mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Fix and rewrite tests
This commit is contained in:
parent
81b5caad2d
commit
2381c3dbca
@ -661,6 +661,7 @@ void ColumnObject::serializePathAndValueIntoArena(DB::Arena & arena, const char
|
||||
|
||||
const char * ColumnObject::deserializeAndInsertFromArena(const char * pos)
|
||||
{
|
||||
size_t current_size = size();
|
||||
/// Deserialize paths and values and insert them into typed paths, dynamic paths or shared data.
|
||||
/// Serialized paths could be unsorted, so we will have to sort all paths that will be inserted into shared data.
|
||||
std::vector<std::pair<StringRef, StringRef>> paths_and_values_for_shared_data;
|
||||
@ -718,6 +719,21 @@ const char * ColumnObject::deserializeAndInsertFromArena(const char * pos)
|
||||
}
|
||||
|
||||
getSharedDataOffsets().push_back(shared_data_paths->size());
|
||||
|
||||
/// Insert default value in all remaining typed and dynamic paths.
|
||||
|
||||
for (auto & [_, column] : typed_paths)
|
||||
{
|
||||
if (column->size() == current_size)
|
||||
column->insertDefault();
|
||||
}
|
||||
|
||||
for (auto & [_, column] : dynamic_paths_ptrs)
|
||||
{
|
||||
if (column->size() == current_size)
|
||||
column->insertDefault();
|
||||
}
|
||||
|
||||
return pos;
|
||||
}
|
||||
|
||||
|
@ -711,6 +711,7 @@ std::optional<NameAndTypePair> ColumnsDescription::tryGetColumn(const GetColumns
|
||||
return NameAndTypePair(ordinary_column_name, dynamic_subcolumn_name, it->type, dynamic_subcolumn_type);
|
||||
}
|
||||
}
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
|
@ -27,9 +27,9 @@
|
||||
</substitution>
|
||||
</substitutions>
|
||||
|
||||
<create_query>CREATE TABLE t_json_1(data JSON) ENGINE = MergeTree ORDER BY tuple()</create_query>
|
||||
<create_query>CREATE TABLE t_json_2(data JSON) ENGINE = MergeTree ORDER BY tuple()</create_query>
|
||||
<create_query>CREATE TABLE t_json_3(data JSON) ENGINE = MergeTree ORDER BY tuple()</create_query>
|
||||
<create_query>CREATE TABLE t_json_1(data Object('json')) ENGINE = MergeTree ORDER BY tuple()</create_query>
|
||||
<create_query>CREATE TABLE t_json_2(data Object('json')) ENGINE = MergeTree ORDER BY tuple()</create_query>
|
||||
<create_query>CREATE TABLE t_json_3(data Object('json')) ENGINE = MergeTree ORDER BY tuple()</create_query>
|
||||
|
||||
<query>INSERT INTO t_json_1 SELECT materialize({json1}) FROM numbers(200000)</query>
|
||||
<query>INSERT INTO t_json_2 SELECT {json2} FROM numbers(100000)</query>
|
||||
|
41
tests/performance/new_json_type.xml
Normal file
41
tests/performance/new_json_type.xml
Normal file
@ -0,0 +1,41 @@
|
||||
<test>
|
||||
<settings>
|
||||
<allow_experimental_json_type>1</allow_experimental_json_type>
|
||||
</settings>
|
||||
|
||||
<!-- Looks like the only way to use json in test's queries is to put them to substitution.
|
||||
Otherwise jsons are interpreted as substitutions themselves -->
|
||||
|
||||
<substitutions>
|
||||
<substitution>
|
||||
<name>json1</name>
|
||||
<values>
|
||||
<value>'{"k1":1, "k2": "some"}'</value>
|
||||
</values>
|
||||
</substitution>
|
||||
<substitution>
|
||||
<name>json2</name>
|
||||
<values>
|
||||
<value>'{"col' || toString(number % 100) || '":' || toString(number) || '}'</value>
|
||||
</values>
|
||||
</substitution>
|
||||
<substitution>
|
||||
<name>json3</name>
|
||||
<values>
|
||||
<value>'{"k1":[{"k2":"aaa","k3":[{"k4":"bbb"},{"k4":"ccc"}]},{"k2":"ddd","k3":[{"k4":"eee"},{"k4":"fff"}]}]}'</value>
|
||||
</values>
|
||||
</substitution>
|
||||
</substitutions>
|
||||
|
||||
<create_query>CREATE TABLE t_json_1(data JSON) ENGINE = MergeTree ORDER BY tuple()</create_query>
|
||||
<create_query>CREATE TABLE t_json_2(data JSON) ENGINE = MergeTree ORDER BY tuple()</create_query>
|
||||
<create_query>CREATE TABLE t_json_3(data JSON) ENGINE = MergeTree ORDER BY tuple()</create_query>
|
||||
|
||||
<query>INSERT INTO t_json_1 SELECT materialize({json1}) FROM numbers(200000)</query>
|
||||
<query>INSERT INTO t_json_2 SELECT {json2} FROM numbers(100000)</query>
|
||||
<query>INSERT INTO t_json_3 SELECT materialize({json3}) FROM numbers_mt(100000)</query>
|
||||
|
||||
<drop_query>DROP TABLE IF EXISTS t_json_1</drop_query>
|
||||
<drop_query>DROP TABLE IF EXISTS t_json_2</drop_query>
|
||||
<drop_query>DROP TABLE IF EXISTS t_json_3</drop_query>
|
||||
</test>
|
@ -20,6 +20,8 @@
|
||||
('k2','String')
|
||||
('k3','String')
|
||||
('k4','Int64')
|
||||
[['{"k2":"aaa","k3":"bbb"}','{"k2":"ccc"}']]
|
||||
[['{"k3":"ddd","k4":"10"}','{"k4":"20"}']]
|
||||
{"arr":[{"x":1}]}
|
||||
{"arr":{"x":{"y":1},"t":{"y":2}}}
|
||||
{"arr":[1,{"y":1}]}
|
||||
|
@ -26,6 +26,8 @@ SELECT id, arr.k1[].k2, arr.k1[].k3, arr.k1[].k4, arr.k5.k6 FROM t_json_array OR
|
||||
SELECT arrayJoin(arrayJoin(arr.k1[])) AS k1 FROM t_json_array ORDER BY toString(k1) FORMAT JSONEachRow;
|
||||
SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(arrayJoin(arrayJoin(arr.k1[])))) AS path FROM t_json_array order by path;
|
||||
|
||||
SELECT arr.k1 FROM t_json_array GROUP BY arr.k1 ORDER BY toString(arr.k1);
|
||||
|
||||
DROP TABLE t_json_array;
|
||||
|
||||
SELECT * FROM values('arr Array(JSON)', '[\'{"x" : 1}\']') FORMAT JSONEachRow;
|
||||
|
@ -17,7 +17,7 @@ ${CLICKHOUSE_CLIENT} -q "SELECT count() FROM nbagames WHERE NOT ignore(*)"
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(data)) as path from nbagames order by path"
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT DISTINCT arrayJoin(JSONAllPathsWithTypes(arrayJoin(data.teams[]))) as path from nbagames order by path"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q \
|
||||
${CLICKHOUSE_CLIENT} --allow_experimental_analyzer=1 -q \
|
||||
"SELECT teams.name.:String AS name, sum(teams.won.:Int64) AS wins FROM nbagames \
|
||||
ARRAY JOIN data.teams[] AS teams GROUP BY name \
|
||||
ORDER BY wins DESC LIMIT 5;"
|
||||
|
@ -1,6 +1,3 @@
|
||||
No merges
|
||||
insert
|
||||
test
|
||||
('a.b.c','UInt32')
|
||||
('a.b.d','Array(Nullable(String))')
|
||||
('a.b.d','DateTime64(9)')
|
||||
@ -414,8 +411,6 @@ test
|
||||
"json.a.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, "10", "11", "12", "13", "14", null, null, null, null, null, "20", "21", "22", "23", "24", "25", "26", "27", "28", "29", null, null, null, null, null, null, null, null, null, null],
|
||||
"json.a.b.d.:`Date`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null]
|
||||
}
|
||||
With merges
|
||||
test
|
||||
('a.b.c','UInt32')
|
||||
('a.b.d','Array(Nullable(String))')
|
||||
('a.b.d','DateTime64(9)')
|
@ -1,107 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest, long
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# reset --log_comment
|
||||
CLICKHOUSE_LOG_COMMENT=
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_json_type=1 --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --session_timezone=UTC"
|
||||
|
||||
function insert()
|
||||
{
|
||||
echo "insert"
|
||||
$CH_CLIENT -q "truncate table test"
|
||||
$CH_CLIENT -q "insert into test select number, '{}' from numbers(5)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number)) from numbers(5, 5)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number))) from numbers(10, 5)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('b.b.d', number::UInt32, 'b.b.e', 'str_' || toString(number))) from numbers(15, 5)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number))) from numbers(20, 5)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number), 'b.b._' || toString(number), number::UInt32)) from numbers(25, 5)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', range(number % + 1)::Array(UInt32), 'a.b.e', 'str_' || toString(number), 'd.a', number::UInt32, 'd.c', toDate(number))) from numbers(30, 5)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', toDateTime(number), 'a.b.e', 'str_' || toString(number), 'd.a', range(number % 5 + 1)::Array(UInt32), 'd.b', number::UInt32)) from numbers(35, 5)"
|
||||
}
|
||||
|
||||
function test()
|
||||
{
|
||||
echo "test"
|
||||
$CH_CLIENT -q "select distinct arrayJoin(JSONAllPathsWithTypes(json)) as paths_with_types from test order by paths_with_types"
|
||||
|
||||
$CH_CLIENT -q "select json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.\`_25\`, json.b.b.\`_25\`.:Int64, json.b.b.\`_25\`.:UUID, json.b.b.\`_26\`, json.b.b.\`_26\`.:Int64, json.b.b.\`_26\`.:UUID, json.b.b.\`_27\`, json.b.b.\`_27\`.:Int64, json.b.b.\`_27\`.:UUID, json.b.b.\`_28\`, json.b.b.\`_28\`.:Int64, json.b.b.\`_28\`.:UUID, json.b.b.\`_29\`, json.b.b.\`_29\`.:Int64, json.b.b.\`_29\`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.\`_25\`, json.b.b.\`_25\`.:Int64, json.b.b.\`_25\`.:UUID, json.b.b.\`_26\`, json.b.b.\`_26\`.:Int64, json.b.b.\`_26\`.:UUID, json.b.b.\`_27\`, json.b.b.\`_27\`.:Int64, json.b.b.\`_27\`.:UUID, json.b.b.\`_28\`, json.b.b.\`_28\`.:Int64, json.b.b.\`_28\`.:UUID, json.b.b.\`_29\`, json.b.b.\`_29\`.:Int64, json.b.b.\`_29\`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.non.existing.path from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.non.existing.path.:Int64 from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.non.existing.path, json.non.existing.path.:Int64 from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path.:Int64 from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path, json.non.existing.path.:Int64 from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.a.b.c from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.a.b.c from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.b.b.e from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.b.b.e, json.a.b.d from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.a.b.d from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.b.b.e, json.d.a from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.d.a from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.b.b.e, json.d.a, json.d.b from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.d.a, json.d.b from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.d.a, json.d.b from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.b from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.d.a, json.b.b.\`_26\` from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_26\`.:Int64, json.b.b.\`_26\`.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_26\`.:Int64, json.b.b, json.b.b.\`_26\`.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.b.b.\`_26\` from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_26\`.:Int64, json.b.b.\`_26\`.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_26\`.:Int64, json.b.b, json.b.b.\`_26\`.:Date from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.^a, json.a.b.c from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.c from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.^a, json.a.b.d from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.d from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns"
|
||||
}
|
||||
|
||||
$CH_CLIENT -q "drop table if exists test;"
|
||||
|
||||
$CH_CLIENT -q "create table test (id UInt64, json JSON(max_dynamic_paths=2, a.b.c UInt32)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;"
|
||||
echo "No merges"
|
||||
$CH_CLIENT -q "system stop merges test"
|
||||
insert
|
||||
test
|
||||
echo "With merges"
|
||||
$CH_CLIENT -q "system start merges test"
|
||||
test
|
||||
$CH_CLIENT -q "drop table test;"
|
@ -0,0 +1,93 @@
|
||||
-- Tags: no-fasttest
|
||||
|
||||
set allow_experimental_json_type = 1;
|
||||
set allow_experimental_variant_type = 1;
|
||||
set use_variant_as_common_type = 1;
|
||||
set session_timezone = 'UTC';
|
||||
|
||||
drop table if exists test;
|
||||
create table test (id UInt64, json JSON(max_dynamic_paths=2, a.b.c UInt32)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;
|
||||
|
||||
insert into test select number, '{}' from numbers(5);
|
||||
insert into test select number, toJSONString(map('a.b.c', number)) from numbers(5, 5);
|
||||
insert into test select number, toJSONString(map('a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number))) from numbers(10, 5);
|
||||
insert into test select number, toJSONString(map('b.b.d', number::UInt32, 'b.b.e', 'str_' || toString(number))) from numbers(15, 5);
|
||||
insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number))) from numbers(20, 5);
|
||||
insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number), 'b.b._' || toString(number), number::UInt32)) from numbers(25, 5);
|
||||
insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', range(number % + 1)::Array(UInt32), 'a.b.e', 'str_' || toString(number), 'd.a', number::UInt32, 'd.c', toDate(number))) from numbers(30, 5);
|
||||
insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', toDateTime(number), 'a.b.e', 'str_' || toString(number), 'd.a', range(number % 5 + 1)::Array(UInt32), 'd.b', number::UInt32)) from numbers(35, 5);
|
||||
|
||||
{% for merge_command in ['system stop merges test', 'system start merges test'] -%}
|
||||
|
||||
{{ merge_command }};
|
||||
|
||||
select distinct arrayJoin(JSONAllPathsWithTypes(json)) as paths_with_types from test order by paths_with_types;
|
||||
|
||||
select json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.`_25`, json.b.b.`_25`.:Int64, json.b.b.`_25`.:UUID, json.b.b.`_26`, json.b.b.`_26`.:Int64, json.b.b.`_26`.:UUID, json.b.b.`_27`, json.b.b.`_27`.:Int64, json.b.b.`_27`.:UUID, json.b.b.`_28`, json.b.b.`_28`.:Int64, json.b.b.`_28`.:UUID, json.b.b.`_29`, json.b.b.`_29`.:Int64, json.b.b.`_29`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test order by id format JSONColumns;
|
||||
select json, json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.`_25`, json.b.b.`_25`.:Int64, json.b.b.`_25`.:UUID, json.b.b.`_26`, json.b.b.`_26`.:Int64, json.b.b.`_26`.:UUID, json.b.b.`_27`, json.b.b.`_27`.:Int64, json.b.b.`_27`.:UUID, json.b.b.`_28`, json.b.b.`_28`.:Int64, json.b.b.`_28`.:UUID, json.b.b.`_29`, json.b.b.`_29`.:Int64, json.b.b.`_29`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test order by id format JSONColumns;
|
||||
|
||||
select json.non.existing.path from test order by id format JSONColumns;
|
||||
select json.non.existing.path.:Int64 from test order by id format JSONColumns;
|
||||
select json.non.existing.path, json.non.existing.path.:Int64 from test order by id format JSONColumns;
|
||||
select json, json.non.existing.path from test order by id format JSONColumns;
|
||||
select json, json.non.existing.path.:Int64 from test order by id format JSONColumns;
|
||||
select json, json.non.existing.path, json.non.existing.path.:Int64 from test order by id format JSONColumns;
|
||||
|
||||
select json.a.b.c from test order by id format JSONColumns;
|
||||
select json, json.a.b.c from test order by id format JSONColumns;
|
||||
|
||||
select json.b.b.e from test order by id format JSONColumns;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date from test order by id format JSONColumns;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test order by id format JSONColumns;
|
||||
select json, json.b.b.e from test order by id format JSONColumns;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date from test order by id format JSONColumns;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test order by id format JSONColumns;
|
||||
|
||||
select json.b.b.e, json.a.b.d from test order by id format JSONColumns;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns;
|
||||
select json, json.b.b.e, json.a.b.d from test order by id format JSONColumns;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns;
|
||||
|
||||
select json.b.b.e, json.d.a from test order by id format JSONColumns;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test order by id format JSONColumns;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test order by id format JSONColumns;
|
||||
select json, json.b.b.e, json.d.a from test order by id format JSONColumns;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test order by id format JSONColumns;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test order by id format JSONColumns;
|
||||
|
||||
select json.b.b.e, json.d.a, json.d.b from test order by id format JSONColumns;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns;
|
||||
select json, json.b.b.e, json.d.a, json.d.b from test order by id format JSONColumns;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns;
|
||||
|
||||
select json.d.a, json.d.b from test order by id format JSONColumns;
|
||||
select json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns;
|
||||
select json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns;
|
||||
select json, json.d.a, json.d.b from test order by id format JSONColumns;
|
||||
select json, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns;
|
||||
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns;
|
||||
|
||||
select json.d.a, json.b.b.`_26` from test order by id format JSONColumns;
|
||||
select json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_26`.:Int64, json.b.b.`_26`.:Date from test order by id format JSONColumns;
|
||||
select json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_26`.:Int64, json.b.b, json.b.b.`_26`.:Date from test order by id format JSONColumns;
|
||||
select json, json.d.a, json.b.b.`_26` from test order by id format JSONColumns;
|
||||
select json, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_26`.:Int64, json.b.b.`_26`.:Date from test order by id format JSONColumns;
|
||||
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_26`.:Int64, json.b.b, json.b.b.`_26`.:Date from test order by id format JSONColumns;
|
||||
|
||||
select json.^a, json.a.b.c from test order by id format JSONColumns;
|
||||
select json, json.^a, json.a.b.c from test order by id format JSONColumns;
|
||||
|
||||
select json.^a, json.a.b.d from test order by id format JSONColumns;
|
||||
select json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns;
|
||||
select json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns;
|
||||
select json, json.^a, json.a.b.d from test order by id format JSONColumns;
|
||||
select json, json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns;
|
||||
select json, json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns;
|
||||
|
||||
{% endfor -%}
|
||||
|
||||
drop table test;
|
@ -1,6 +1,3 @@
|
||||
No merges
|
||||
insert
|
||||
test
|
||||
('a.b.c','UInt32')
|
||||
('a.b.d','Array(Nullable(String))')
|
||||
('a.b.d','DateTime64(9)')
|
||||
@ -414,8 +411,6 @@ test
|
||||
"json.a.b.d.:`Int64`": [null, null, null, null, null, null, null, null, null, null, "10", "11", "12", "13", "14", null, null, null, null, null, "20", "21", "22", "23", "24", "25", "26", "27", "28", "29", null, null, null, null, null, null, null, null, null, null],
|
||||
"json.a.b.d.:`Date`": [null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null]
|
||||
}
|
||||
With merges
|
||||
test
|
||||
('a.b.c','UInt32')
|
||||
('a.b.d','Array(Nullable(String))')
|
||||
('a.b.d','DateTime64(9)')
|
@ -1,107 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest, long
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# reset --log_comment
|
||||
CLICKHOUSE_LOG_COMMENT=
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_json_type=1 --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --session_timezone=UTC"
|
||||
|
||||
function insert()
|
||||
{
|
||||
echo "insert"
|
||||
$CH_CLIENT -q "truncate table test"
|
||||
$CH_CLIENT -q "insert into test select number, '{}' from numbers(5)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number)) from numbers(5, 5)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number))) from numbers(10, 5)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('b.b.d', number::UInt32, 'b.b.e', 'str_' || toString(number))) from numbers(15, 5)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number))) from numbers(20, 5)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number), 'b.b._' || toString(number), number::UInt32)) from numbers(25, 5)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', range(number % + 1)::Array(UInt32), 'a.b.e', 'str_' || toString(number), 'd.a', number::UInt32, 'd.c', toDate(number))) from numbers(30, 5)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', toDateTime(number), 'a.b.e', 'str_' || toString(number), 'd.a', range(number % 5 + 1)::Array(UInt32), 'd.b', number::UInt32)) from numbers(35, 5)"
|
||||
}
|
||||
|
||||
function test()
|
||||
{
|
||||
echo "test"
|
||||
$CH_CLIENT -q "select distinct arrayJoin(JSONAllPathsWithTypes(json)) as paths_with_types from test order by paths_with_types"
|
||||
|
||||
$CH_CLIENT -q "select json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.\`_25\`, json.b.b.\`_25\`.:Int64, json.b.b.\`_25\`.:UUID, json.b.b.\`_26\`, json.b.b.\`_26\`.:Int64, json.b.b.\`_26\`.:UUID, json.b.b.\`_27\`, json.b.b.\`_27\`.:Int64, json.b.b.\`_27\`.:UUID, json.b.b.\`_28\`, json.b.b.\`_28\`.:Int64, json.b.b.\`_28\`.:UUID, json.b.b.\`_29\`, json.b.b.\`_29\`.:Int64, json.b.b.\`_29\`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.\`_25\`, json.b.b.\`_25\`.:Int64, json.b.b.\`_25\`.:UUID, json.b.b.\`_26\`, json.b.b.\`_26\`.:Int64, json.b.b.\`_26\`.:UUID, json.b.b.\`_27\`, json.b.b.\`_27\`.:Int64, json.b.b.\`_27\`.:UUID, json.b.b.\`_28\`, json.b.b.\`_28\`.:Int64, json.b.b.\`_28\`.:UUID, json.b.b.\`_29\`, json.b.b.\`_29\`.:Int64, json.b.b.\`_29\`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.non.existing.path from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.non.existing.path.:Int64 from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.non.existing.path, json.non.existing.path.:Int64 from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path.:Int64 from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path, json.non.existing.path.:Int64 from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.a.b.c from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.a.b.c from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.b.b.e from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.b.b.e, json.a.b.d from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.a.b.d from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.b.b.e, json.d.a from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.d.a from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.b.b.e, json.d.a, json.d.b from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.d.a, json.d.b from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.d.a, json.d.b from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.b from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.d.a, json.b.b.\`_26\` from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_26\`.:Int64, json.b.b.\`_26\`.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_26\`.:Int64, json.b.b, json.b.b.\`_26\`.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.b.b.\`_26\` from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_26\`.:Int64, json.b.b.\`_26\`.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_26\`.:Int64, json.b.b, json.b.b.\`_26\`.:Date from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.^a, json.a.b.c from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.c from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.^a, json.a.b.d from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.d from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns"
|
||||
}
|
||||
|
||||
$CH_CLIENT -q "drop table if exists test;"
|
||||
|
||||
$CH_CLIENT -q "create table test (id UInt64, json JSON(max_dynamic_paths=2, a.b.c UInt32)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;"
|
||||
echo "No merges"
|
||||
$CH_CLIENT -q "system stop merges test"
|
||||
insert
|
||||
test
|
||||
echo "With merges"
|
||||
$CH_CLIENT -q "system start merges test"
|
||||
test
|
||||
$CH_CLIENT -q "drop table test;"
|
@ -0,0 +1,93 @@
|
||||
-- Tags: no-fasttest
|
||||
|
||||
set allow_experimental_json_type = 1;
|
||||
set allow_experimental_variant_type = 1;
|
||||
set use_variant_as_common_type = 1;
|
||||
set session_timezone = 'UTC';
|
||||
|
||||
drop table if exists test;
|
||||
create table test (id UInt64, json JSON(max_dynamic_paths=2, a.b.c UInt32)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;
|
||||
|
||||
insert into test select number, '{}' from numbers(5);
|
||||
insert into test select number, toJSONString(map('a.b.c', number)) from numbers(5, 5);
|
||||
insert into test select number, toJSONString(map('a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number))) from numbers(10, 5);
|
||||
insert into test select number, toJSONString(map('b.b.d', number::UInt32, 'b.b.e', 'str_' || toString(number))) from numbers(15, 5);
|
||||
insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number))) from numbers(20, 5);
|
||||
insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number), 'b.b._' || toString(number), number::UInt32)) from numbers(25, 5);
|
||||
insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', range(number % + 1)::Array(UInt32), 'a.b.e', 'str_' || toString(number), 'd.a', number::UInt32, 'd.c', toDate(number))) from numbers(30, 5);
|
||||
insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', toDateTime(number), 'a.b.e', 'str_' || toString(number), 'd.a', range(number % 5 + 1)::Array(UInt32), 'd.b', number::UInt32)) from numbers(35, 5);
|
||||
|
||||
{% for merge_command in ['system stop merges test', 'system start merges test'] -%}
|
||||
|
||||
{{ merge_command }};
|
||||
|
||||
select distinct arrayJoin(JSONAllPathsWithTypes(json)) as paths_with_types from test order by paths_with_types;
|
||||
|
||||
select json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.`_25`, json.b.b.`_25`.:Int64, json.b.b.`_25`.:UUID, json.b.b.`_26`, json.b.b.`_26`.:Int64, json.b.b.`_26`.:UUID, json.b.b.`_27`, json.b.b.`_27`.:Int64, json.b.b.`_27`.:UUID, json.b.b.`_28`, json.b.b.`_28`.:Int64, json.b.b.`_28`.:UUID, json.b.b.`_29`, json.b.b.`_29`.:Int64, json.b.b.`_29`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test order by id format JSONColumns;
|
||||
select json, json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.`_25`, json.b.b.`_25`.:Int64, json.b.b.`_25`.:UUID, json.b.b.`_26`, json.b.b.`_26`.:Int64, json.b.b.`_26`.:UUID, json.b.b.`_27`, json.b.b.`_27`.:Int64, json.b.b.`_27`.:UUID, json.b.b.`_28`, json.b.b.`_28`.:Int64, json.b.b.`_28`.:UUID, json.b.b.`_29`, json.b.b.`_29`.:Int64, json.b.b.`_29`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test order by id format JSONColumns;
|
||||
|
||||
select json.non.existing.path from test order by id format JSONColumns;
|
||||
select json.non.existing.path.:Int64 from test order by id format JSONColumns;
|
||||
select json.non.existing.path, json.non.existing.path.:Int64 from test order by id format JSONColumns;
|
||||
select json, json.non.existing.path from test order by id format JSONColumns;
|
||||
select json, json.non.existing.path.:Int64 from test order by id format JSONColumns;
|
||||
select json, json.non.existing.path, json.non.existing.path.:Int64 from test order by id format JSONColumns;
|
||||
|
||||
select json.a.b.c from test order by id format JSONColumns;
|
||||
select json, json.a.b.c from test order by id format JSONColumns;
|
||||
|
||||
select json.b.b.e from test order by id format JSONColumns;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date from test order by id format JSONColumns;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test order by id format JSONColumns;
|
||||
select json, json.b.b.e from test order by id format JSONColumns;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date from test order by id format JSONColumns;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test order by id format JSONColumns;
|
||||
|
||||
select json.b.b.e, json.a.b.d from test order by id format JSONColumns;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns;
|
||||
select json, json.b.b.e, json.a.b.d from test order by id format JSONColumns;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns;
|
||||
|
||||
select json.b.b.e, json.d.a from test order by id format JSONColumns;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test order by id format JSONColumns;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test order by id format JSONColumns;
|
||||
select json, json.b.b.e, json.d.a from test order by id format JSONColumns;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test order by id format JSONColumns;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test order by id format JSONColumns;
|
||||
|
||||
select json.b.b.e, json.d.a, json.d.b from test order by id format JSONColumns;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns;
|
||||
select json, json.b.b.e, json.d.a, json.d.b from test order by id format JSONColumns;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns;
|
||||
|
||||
select json.d.a, json.d.b from test order by id format JSONColumns;
|
||||
select json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns;
|
||||
select json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns;
|
||||
select json, json.d.a, json.d.b from test order by id format JSONColumns;
|
||||
select json, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns;
|
||||
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format JSONColumns;
|
||||
|
||||
select json.d.a, json.b.b.`_26` from test order by id format JSONColumns;
|
||||
select json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_26`.:Int64, json.b.b.`_26`.:Date from test order by id format JSONColumns;
|
||||
select json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_26`.:Int64, json.b.b, json.b.b.`_26`.:Date from test order by id format JSONColumns;
|
||||
select json, json.d.a, json.b.b.`_26` from test order by id format JSONColumns;
|
||||
select json, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_26`.:Int64, json.b.b.`_26`.:Date from test order by id format JSONColumns;
|
||||
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_26`.:Int64, json.b.b, json.b.b.`_26`.:Date from test order by id format JSONColumns;
|
||||
|
||||
select json.^a, json.a.b.c from test order by id format JSONColumns;
|
||||
select json, json.^a, json.a.b.c from test order by id format JSONColumns;
|
||||
|
||||
select json.^a, json.a.b.d from test order by id format JSONColumns;
|
||||
select json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns;
|
||||
select json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns;
|
||||
select json, json.^a, json.a.b.d from test order by id format JSONColumns;
|
||||
select json, json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns;
|
||||
select json, json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format JSONColumns;
|
||||
|
||||
{% endfor -%}
|
||||
|
||||
drop table test;
|
@ -1,6 +1,3 @@
|
||||
No merges
|
||||
insert
|
||||
test
|
||||
('a.b.c','UInt32')
|
||||
('a.b.d','Array(Nullable(String))')
|
||||
('a.b.d','DateTime64(9)')
|
||||
@ -34,8 +31,6 @@ test
|
||||
680000
|
||||
0
|
||||
0
|
||||
With merges
|
||||
test
|
||||
('a.b.c','UInt32')
|
||||
('a.b.d','Array(Nullable(String))')
|
||||
('a.b.d','DateTime64(9)')
|
@ -1,142 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest, long, no-debug, no-tsan, no-asan, no-msan, no-ubsan
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# reset --log_comment
|
||||
CLICKHOUSE_LOG_COMMENT=
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_json_type=1 --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --session_timezone=UTC"
|
||||
|
||||
function insert()
|
||||
{
|
||||
echo "insert"
|
||||
$CH_CLIENT -q "truncate table test"
|
||||
$CH_CLIENT -q "insert into test select number, '{}' from numbers(100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number)) from numbers(100000, 100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number))) from numbers(200000, 100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('b.b.d', number::UInt32, 'b.b.e', 'str_' || toString(number))) from numbers(300000, 100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number))) from numbers(400000, 100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number), 'b.b._' || toString(number % 5), number::UInt32)) from numbers(500000, 100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', range(number % + 1)::Array(UInt32), 'a.b.e', 'str_' || toString(number), 'd.a', number::UInt32, 'd.c', toDate(number))) from numbers(600000, 100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', toDateTime(number), 'a.b.e', 'str_' || toString(number), 'd.a', range(number % 5 + 1)::Array(UInt32), 'd.b', number::UInt32)) from numbers(700000, 100000)"
|
||||
}
|
||||
|
||||
function test()
|
||||
{
|
||||
echo "test"
|
||||
$CH_CLIENT -q "select distinct arrayJoin(JSONAllPathsWithTypes(json)) as paths_with_types from test order by paths_with_types"
|
||||
|
||||
$CH_CLIENT -q "select json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.\`_0\`, json.b.b.\`_0\`.:Int64, json.b.b.\`_0\`.:UUID, json.b.b.\`_1\`, json.b.b.\`_1\`.:Int64, json.b.b.\`_1\`.:UUID, json.b.b.\`_2\`, json.b.b.\`_2\`.:Int64, json.b.b.\`_2\`.:UUID, json.b.b.\`_3\`, json.b.b.\`_3\`.:Int64, json.b.b.\`_3\`.:UUID, json.b.b.\`_4\`, json.b.b.\`_4\`.:Int64, json.b.b.\`_4\`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test format Null"
|
||||
$CH_CLIENT -q "select json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.\`_0\`, json.b.b.\`_0\`.:Int64, json.b.b.\`_0\`.:UUID, json.b.b.\`_1\`, json.b.b.\`_1\`.:Int64, json.b.b.\`_1\`.:UUID, json.b.b.\`_2\`, json.b.b.\`_2\`.:Int64, json.b.b.\`_2\`.:UUID, json.b.b.\`_3\`, json.b.b.\`_3\`.:Int64, json.b.b.\`_3\`.:UUID, json.b.b.\`_4\`, json.b.b.\`_4\`.:Int64, json.b.b.\`_4\`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.\`_0\`, json.b.b.\`_0\`.:Int64, json.b.b.\`_0\`.:UUID, json.b.b.\`_1\`, json.b.b.\`_1\`.:Int64, json.b.b.\`_1\`.:UUID, json.b.b.\`_2\`, json.b.b.\`_2\`.:Int64, json.b.b.\`_2\`.:UUID, json.b.b.\`_3\`, json.b.b.\`_3\`.:Int64, json.b.b.\`_3\`.:UUID, json.b.b.\`_4\`, json.b.b.\`_4\`.:Int64, json.b.b.\`_4\`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test format Null"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.\`_0\`, json.b.b.\`_0\`.:Int64, json.b.b.\`_0\`.:UUID, json.b.b.\`_1\`, json.b.b.\`_1\`.:Int64, json.b.b.\`_1\`.:UUID, json.b.b.\`_2\`, json.b.b.\`_2\`.:Int64, json.b.b.\`_2\`.:UUID, json.b.b.\`_3\`, json.b.b.\`_3\`.:Int64, json.b.b.\`_3\`.:UUID, json.b.b.\`_4\`, json.b.b.\`_4\`.:Int64, json.b.b.\`_4\`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.non.existing.path is Null"
|
||||
$CH_CLIENT -q "select count() from test where json.non.existing.path.:String is Null"
|
||||
$CH_CLIENT -q "select json.non.existing.path from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.non.existing.path.:Int64 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.non.existing.path, json.non.existing.path.:Int64 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path.:Int64 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path, json.non.existing.path.:Int64 from test format Null"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path, json.non.existing.path.:Int64 from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.a.b.c == 0"
|
||||
$CH_CLIENT -q "select json.a.b.c from test format Null"
|
||||
$CH_CLIENT -q "select json.a.b.c from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.a.b.c from test format Null"
|
||||
$CH_CLIENT -q "select json, json.a.b.c from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e is Null"
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e.:String is Null"
|
||||
$CH_CLIENT -q "select json.b.b.e from test format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e from test format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e is Null and json.a.b.d is Null "
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e.:String is Null and json.a.b.d.:Int64 is Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.a.b.d from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.a.b.d from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e is Null and json.d.a is Null"
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e.:String is Null and empty(json.d.a.:\`Array(Nullable(Int64))\`)"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.d.a from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.d.a from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e is Null and json.d.a is Null and json.d.b is Null"
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e.:String is Null and empty(json.d.a.:\`Array(Nullable(Int64))\`) and json.d.b.:Int64 is Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.d.a, json.d.b from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.d.a, json.d.b from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.d.a is Null and json.d.b is Null"
|
||||
$CH_CLIENT -q "select count() from test where empty(json.d.a.:\`Array(Nullable(Int64))\`) and json.d.b.:Int64 is Null"
|
||||
$CH_CLIENT -q "select json.d.a, json.d.b from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.b from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.d.a is Null and json.b.b.\`_1\` is Null"
|
||||
$CH_CLIENT -q "select count() from test where empty(json.d.a.:\`Array(Nullable(Int64))\`) and json.b.b.\`_1\`.:Int64 is Null"
|
||||
$CH_CLIENT -q "select json.d.a, json.b.b.\`_1\` from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_1\`.:Int64, json.b.b.\`_1\`.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_1\`.:Int64, json.b.b, json.b.b.\`_1\`.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.b.b.\`_1\` from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_1\`.:Int64, json.b.b.\`_1\`.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_1\`.:Int64, json.b.b, json.b.b.\`_1\`.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_1\`.:Int64, json.b.b, json.b.b.\`_1\`.:Date from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where JSONEmpty(json.^a) and json.a.b.c == 0"
|
||||
$CH_CLIENT -q "select json.^a, json.a.b.c from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.c from test format Null"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.c from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where JSONEmpty(json.^a) and json.a.b.d is Null"
|
||||
$CH_CLIENT -q "select json.^a, json.a.b.d from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.d from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
}
|
||||
|
||||
$CH_CLIENT -q "drop table if exists test;"
|
||||
|
||||
$CH_CLIENT -q "create table test (id UInt64, json JSON(max_dynamic_paths=2, a.b.c UInt32)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;"
|
||||
echo "No merges"
|
||||
$CH_CLIENT -q "system stop merges test"
|
||||
insert
|
||||
test
|
||||
echo "With merges"
|
||||
$CH_CLIENT -q "system start merges test"
|
||||
test
|
||||
$CH_CLIENT -q "drop table test;"
|
@ -0,0 +1,128 @@
|
||||
-- Tags: no-fasttest, long, no-debug, no-tsan, no-asan, no-msan, no-ubsan
|
||||
|
||||
set allow_experimental_json_type = 1;
|
||||
set allow_experimental_variant_type = 1;
|
||||
set use_variant_as_common_type = 1;
|
||||
set session_timezone = 'UTC';
|
||||
|
||||
drop table if exists test;
|
||||
create table test (id UInt64, json JSON(max_dynamic_paths=2, a.b.c UInt32)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;
|
||||
|
||||
insert into test select number, '{}' from numbers(100000);
|
||||
insert into test select number, toJSONString(map('a.b.c', number)) from numbers(100000, 100000);
|
||||
insert into test select number, toJSONString(map('a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number))) from numbers(200000, 100000);
|
||||
insert into test select number, toJSONString(map('b.b.d', number::UInt32, 'b.b.e', 'str_' || toString(number))) from numbers(300000, 100000);
|
||||
insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number))) from numbers(400000, 100000);
|
||||
insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number), 'b.b._' || toString(number % 5), number::UInt32)) from numbers(500000, 100000);
|
||||
insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', range(number % + 1)::Array(UInt32), 'a.b.e', 'str_' || toString(number), 'd.a', number::UInt32, 'd.c', toDate(number))) from numbers(600000, 100000);
|
||||
insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', toDateTime(number), 'a.b.e', 'str_' || toString(number), 'd.a', range(number % 5 + 1)::Array(UInt32), 'd.b', number::UInt32)) from numbers(700000, 100000);
|
||||
|
||||
{% for merge_command in ['system stop merges test', 'system start merges test'] -%}
|
||||
|
||||
{{ merge_command }};
|
||||
|
||||
select distinct arrayJoin(JSONAllPathsWithTypes(json)) as paths_with_types from test order by paths_with_types;
|
||||
|
||||
select json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.`_0`, json.b.b.`_0`.:Int64, json.b.b.`_0`.:UUID, json.b.b.`_1`, json.b.b.`_1`.:Int64, json.b.b.`_1`.:UUID, json.b.b.`_2`, json.b.b.`_2`.:Int64, json.b.b.`_2`.:UUID, json.b.b.`_3`, json.b.b.`_3`.:Int64, json.b.b.`_3`.:UUID, json.b.b.`_4`, json.b.b.`_4`.:Int64, json.b.b.`_4`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test format Null;
|
||||
select json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.`_0`, json.b.b.`_0`.:Int64, json.b.b.`_0`.:UUID, json.b.b.`_1`, json.b.b.`_1`.:Int64, json.b.b.`_1`.:UUID, json.b.b.`_2`, json.b.b.`_2`.:Int64, json.b.b.`_2`.:UUID, json.b.b.`_3`, json.b.b.`_3`.:Int64, json.b.b.`_3`.:UUID, json.b.b.`_4`, json.b.b.`_4`.:Int64, json.b.b.`_4`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test order by id format Null;
|
||||
select json, json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.`_0`, json.b.b.`_0`.:Int64, json.b.b.`_0`.:UUID, json.b.b.`_1`, json.b.b.`_1`.:Int64, json.b.b.`_1`.:UUID, json.b.b.`_2`, json.b.b.`_2`.:Int64, json.b.b.`_2`.:UUID, json.b.b.`_3`, json.b.b.`_3`.:Int64, json.b.b.`_3`.:UUID, json.b.b.`_4`, json.b.b.`_4`.:Int64, json.b.b.`_4`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test format Null;
|
||||
select json, json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.`_0`, json.b.b.`_0`.:Int64, json.b.b.`_0`.:UUID, json.b.b.`_1`, json.b.b.`_1`.:Int64, json.b.b.`_1`.:UUID, json.b.b.`_2`, json.b.b.`_2`.:Int64, json.b.b.`_2`.:UUID, json.b.b.`_3`, json.b.b.`_3`.:Int64, json.b.b.`_3`.:UUID, json.b.b.`_4`, json.b.b.`_4`.:Int64, json.b.b.`_4`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test order by id format Null;
|
||||
|
||||
select count() from test where json.non.existing.path is Null;
|
||||
select count() from test where json.non.existing.path.:String is Null;
|
||||
select json.non.existing.path from test order by id format Null;
|
||||
select json.non.existing.path.:Int64 from test order by id format Null;
|
||||
select json.non.existing.path, json.non.existing.path.:Int64 from test order by id format Null;
|
||||
select json, json.non.existing.path from test order by id format Null;
|
||||
select json, json.non.existing.path.:Int64 from test order by id format Null;
|
||||
select json, json.non.existing.path, json.non.existing.path.:Int64 from test format Null;
|
||||
select json, json.non.existing.path, json.non.existing.path.:Int64 from test order by id format Null;
|
||||
|
||||
select count() from test where json.a.b.c == 0;
|
||||
select json.a.b.c from test format Null;
|
||||
select json.a.b.c from test order by id format Null;
|
||||
select json, json.a.b.c from test format Null;
|
||||
select json, json.a.b.c from test order by id format Null;
|
||||
|
||||
select count() from test where json.b.b.e is Null;
|
||||
select count() from test where json.b.b.e.:String is Null;
|
||||
select json.b.b.e from test format Null;
|
||||
select json.b.b.e from test order by id format Null;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date from test format Null;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test format Null;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null;
|
||||
select json, json.b.b.e from test format Null;
|
||||
select json, json.b.b.e from test order by id format Null;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date from test format Null;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null;
|
||||
|
||||
select count() from test where json.b.b.e is Null and json.a.b.d is Null ;
|
||||
select count() from test where json.b.b.e.:String is Null and json.a.b.d.:Int64 is Null;
|
||||
select json.b.b.e, json.a.b.d from test order by id format Null;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.a.b.d from test order by id format Null;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
|
||||
select count() from test where json.b.b.e is Null and json.d.a is Null;
|
||||
select count() from test where json.b.b.e.:String is Null and empty(json.d.a.:`Array(Nullable(Int64))`);
|
||||
select json.b.b.e, json.d.a from test order by id format Null;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test order by id format Null;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.d.a from test order by id format Null;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test order by id format Null;
|
||||
|
||||
select count() from test where json.b.b.e is Null and json.d.a is Null and json.d.b is Null;
|
||||
select count() from test where json.b.b.e.:String is Null and empty(json.d.a.:`Array(Nullable(Int64))`) and json.d.b.:Int64 is Null;
|
||||
select json.b.b.e, json.d.a, json.d.b from test order by id format Null;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.d.a, json.d.b from test order by id format Null;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
|
||||
select count() from test where json.d.a is Null and json.d.b is Null;
|
||||
select count() from test where empty(json.d.a.:`Array(Nullable(Int64))`) and json.d.b.:Int64 is Null;
|
||||
select json.d.a, json.d.b from test order by id format Null;
|
||||
select json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
select json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
select json, json.d.a, json.d.b from test order by id format Null;
|
||||
select json, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test format Null;
|
||||
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
|
||||
select count() from test where json.d.a is Null and json.b.b.`_1` is Null;
|
||||
select count() from test where empty(json.d.a.:`Array(Nullable(Int64))`) and json.b.b.`_1`.:Int64 is Null;
|
||||
select json.d.a, json.b.b.`_1` from test order by id format Null;
|
||||
select json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b.`_1`.:Date from test order by id format Null;
|
||||
select json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b, json.b.b.`_1`.:Date from test order by id format Null;
|
||||
select json, json.d.a, json.b.b.`_1` from test order by id format Null;
|
||||
select json, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b.`_1`.:Date from test order by id format Null;
|
||||
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b, json.b.b.`_1`.:Date from test format Null;
|
||||
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b, json.b.b.`_1`.:Date from test order by id format Null;
|
||||
|
||||
select count() from test where JSONEmpty(json.^a) and json.a.b.c == 0;
|
||||
select json.^a, json.a.b.c from test order by id format Null;
|
||||
select json, json.^a, json.a.b.c from test format Null;
|
||||
select json, json.^a, json.a.b.c from test order by id format Null;
|
||||
|
||||
select count() from test where JSONEmpty(json.^a) and json.a.b.d is Null;
|
||||
select json.^a, json.a.b.d from test order by id format Null;
|
||||
select json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
select json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
select json, json.^a, json.a.b.d from test order by id format Null;
|
||||
select json, json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
select json, json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test format Null;
|
||||
select json, json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
|
||||
{% endfor -%}
|
||||
|
||||
drop table test;
|
@ -1,5 +1,3 @@
|
||||
insert
|
||||
test
|
||||
('a.b.c','UInt32')
|
||||
('a.b.d','Array(Nullable(String))')
|
||||
('a.b.d','DateTime64(9)')
|
||||
|
@ -1,137 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest, long, no-debug, no-tsan, no-asan, no-msan, no-ubsan
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# reset --log_comment
|
||||
CLICKHOUSE_LOG_COMMENT=
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_json_type=1 --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --session_timezone=UTC"
|
||||
|
||||
function insert()
|
||||
{
|
||||
echo "insert"
|
||||
$CH_CLIENT -q "truncate table test"
|
||||
$CH_CLIENT -q "insert into test select number, '{}' from numbers(100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number)) from numbers(100000, 100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number))) from numbers(200000, 100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('b.b.d', number::UInt32, 'b.b.e', 'str_' || toString(number))) from numbers(300000, 100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number))) from numbers(400000, 100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number), 'b.b._' || toString(number % 5), number::UInt32)) from numbers(500000, 100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', range(number % + 1)::Array(UInt32), 'a.b.e', 'str_' || toString(number), 'd.a', number::UInt32, 'd.c', toDate(number))) from numbers(600000, 100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', toDateTime(number), 'a.b.e', 'str_' || toString(number), 'd.a', range(number % 5 + 1)::Array(UInt32), 'd.b', number::UInt32)) from numbers(700000, 100000)"
|
||||
}
|
||||
|
||||
function test()
|
||||
{
|
||||
echo "test"
|
||||
$CH_CLIENT -q "select distinct arrayJoin(JSONAllPathsWithTypes(json)) as paths_with_types from test order by paths_with_types"
|
||||
|
||||
$CH_CLIENT -q "select json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.\`_0\`, json.b.b.\`_0\`.:Int64, json.b.b.\`_0\`.:UUID, json.b.b.\`_1\`, json.b.b.\`_1\`.:Int64, json.b.b.\`_1\`.:UUID, json.b.b.\`_2\`, json.b.b.\`_2\`.:Int64, json.b.b.\`_2\`.:UUID, json.b.b.\`_3\`, json.b.b.\`_3\`.:Int64, json.b.b.\`_3\`.:UUID, json.b.b.\`_4\`, json.b.b.\`_4\`.:Int64, json.b.b.\`_4\`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test format Null"
|
||||
$CH_CLIENT -q "select json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.\`_0\`, json.b.b.\`_0\`.:Int64, json.b.b.\`_0\`.:UUID, json.b.b.\`_1\`, json.b.b.\`_1\`.:Int64, json.b.b.\`_1\`.:UUID, json.b.b.\`_2\`, json.b.b.\`_2\`.:Int64, json.b.b.\`_2\`.:UUID, json.b.b.\`_3\`, json.b.b.\`_3\`.:Int64, json.b.b.\`_3\`.:UUID, json.b.b.\`_4\`, json.b.b.\`_4\`.:Int64, json.b.b.\`_4\`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.\`_0\`, json.b.b.\`_0\`.:Int64, json.b.b.\`_0\`.:UUID, json.b.b.\`_1\`, json.b.b.\`_1\`.:Int64, json.b.b.\`_1\`.:UUID, json.b.b.\`_2\`, json.b.b.\`_2\`.:Int64, json.b.b.\`_2\`.:UUID, json.b.b.\`_3\`, json.b.b.\`_3\`.:Int64, json.b.b.\`_3\`.:UUID, json.b.b.\`_4\`, json.b.b.\`_4\`.:Int64, json.b.b.\`_4\`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test format Null"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.\`_0\`, json.b.b.\`_0\`.:Int64, json.b.b.\`_0\`.:UUID, json.b.b.\`_1\`, json.b.b.\`_1\`.:Int64, json.b.b.\`_1\`.:UUID, json.b.b.\`_2\`, json.b.b.\`_2\`.:Int64, json.b.b.\`_2\`.:UUID, json.b.b.\`_3\`, json.b.b.\`_3\`.:Int64, json.b.b.\`_3\`.:UUID, json.b.b.\`_4\`, json.b.b.\`_4\`.:Int64, json.b.b.\`_4\`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.non.existing.path is Null"
|
||||
$CH_CLIENT -q "select count() from test where json.non.existing.path.:String is Null"
|
||||
$CH_CLIENT -q "select json.non.existing.path from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.non.existing.path.:Int64 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.non.existing.path, json.non.existing.path.:Int64 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path.:Int64 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path, json.non.existing.path.:Int64 from test format Null"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path, json.non.existing.path.:Int64 from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.a.b.c == 0"
|
||||
$CH_CLIENT -q "select json.a.b.c from test format Null"
|
||||
$CH_CLIENT -q "select json.a.b.c from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.a.b.c from test format Null"
|
||||
$CH_CLIENT -q "select json, json.a.b.c from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e is Null"
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e.:String is Null"
|
||||
$CH_CLIENT -q "select json.b.b.e from test format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e from test format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e is Null and json.a.b.d is Null "
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e.:String is Null and json.a.b.d.:Int64 is Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.a.b.d from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.a.b.d from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e is Null and json.d.a is Null"
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e.:String is Null and empty(json.d.a.:\`Array(Nullable(Int64))\`)"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.d.a from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.d.a from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e is Null and json.d.a is Null and json.d.b is Null"
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e.:String is Null and empty(json.d.a.:\`Array(Nullable(Int64))\`) and json.d.b.:Int64 is Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.d.a, json.d.b from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.d.a, json.d.b from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.d.a is Null and json.d.b is Null"
|
||||
$CH_CLIENT -q "select count() from test where empty(json.d.a.:\`Array(Nullable(Int64))\`) and json.d.b.:Int64 is Null"
|
||||
$CH_CLIENT -q "select json.d.a, json.d.b from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.b from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.d.a is Null and json.b.b.\`_1\` is Null"
|
||||
$CH_CLIENT -q "select count() from test where empty(json.d.a.:\`Array(Nullable(Int64))\`) and json.b.b.\`_1\`.:Int64 is Null"
|
||||
$CH_CLIENT -q "select json.d.a, json.b.b.\`_1\` from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_1\`.:Int64, json.b.b.\`_1\`.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_1\`.:Int64, json.b.b, json.b.b.\`_1\`.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.b.b.\`_1\` from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_1\`.:Int64, json.b.b.\`_1\`.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_1\`.:Int64, json.b.b, json.b.b.\`_1\`.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_1\`.:Int64, json.b.b, json.b.b.\`_1\`.:Date from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where JSONEmpty(json.^a) and json.a.b.c == 0"
|
||||
$CH_CLIENT -q "select json.^a, json.a.b.c from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.c from test format Null"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.c from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where JSONEmpty(json.^a) and json.a.b.d is Null"
|
||||
$CH_CLIENT -q "select json.^a, json.a.b.d from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.d from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
}
|
||||
|
||||
$CH_CLIENT -q "drop table if exists test;"
|
||||
|
||||
$CH_CLIENT -q "create table test (id UInt64, json JSON(max_dynamic_paths=2, a.b.c UInt32)) engine=Memory"
|
||||
insert
|
||||
test
|
||||
$CH_CLIENT -q "drop table test;"
|
@ -0,0 +1,123 @@
|
||||
-- Tags: no-fasttest, long, no-debug, no-tsan, no-asan, no-msan, no-ubsan
|
||||
|
||||
set allow_experimental_json_type = 1;
|
||||
set allow_experimental_variant_type = 1;
|
||||
set use_variant_as_common_type = 1;
|
||||
set session_timezone = 'UTC';
|
||||
|
||||
drop table if exists test;
|
||||
create table test (id UInt64, json JSON(max_dynamic_paths=2, a.b.c UInt32)) engine=Memory;
|
||||
|
||||
truncate table test;
|
||||
insert into test select number, '{}' from numbers(100000);
|
||||
insert into test select number, toJSONString(map('a.b.c', number)) from numbers(100000, 100000);
|
||||
insert into test select number, toJSONString(map('a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number))) from numbers(200000, 100000);
|
||||
insert into test select number, toJSONString(map('b.b.d', number::UInt32, 'b.b.e', 'str_' || toString(number))) from numbers(300000, 100000);
|
||||
insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number))) from numbers(400000, 100000);
|
||||
insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number), 'b.b._' || toString(number % 5), number::UInt32)) from numbers(500000, 100000);
|
||||
insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', range(number % + 1)::Array(UInt32), 'a.b.e', 'str_' || toString(number), 'd.a', number::UInt32, 'd.c', toDate(number))) from numbers(600000, 100000);
|
||||
insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', toDateTime(number), 'a.b.e', 'str_' || toString(number), 'd.a', range(number % 5 + 1)::Array(UInt32), 'd.b', number::UInt32)) from numbers(700000, 100000);
|
||||
|
||||
select distinct arrayJoin(JSONAllPathsWithTypes(json)) as paths_with_types from test order by paths_with_types;
|
||||
|
||||
select json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.`_0`, json.b.b.`_0`.:Int64, json.b.b.`_0`.:UUID, json.b.b.`_1`, json.b.b.`_1`.:Int64, json.b.b.`_1`.:UUID, json.b.b.`_2`, json.b.b.`_2`.:Int64, json.b.b.`_2`.:UUID, json.b.b.`_3`, json.b.b.`_3`.:Int64, json.b.b.`_3`.:UUID, json.b.b.`_4`, json.b.b.`_4`.:Int64, json.b.b.`_4`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test format Null;
|
||||
select json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.`_0`, json.b.b.`_0`.:Int64, json.b.b.`_0`.:UUID, json.b.b.`_1`, json.b.b.`_1`.:Int64, json.b.b.`_1`.:UUID, json.b.b.`_2`, json.b.b.`_2`.:Int64, json.b.b.`_2`.:UUID, json.b.b.`_3`, json.b.b.`_3`.:Int64, json.b.b.`_3`.:UUID, json.b.b.`_4`, json.b.b.`_4`.:Int64, json.b.b.`_4`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test order by id format Null;
|
||||
select json, json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.`_0`, json.b.b.`_0`.:Int64, json.b.b.`_0`.:UUID, json.b.b.`_1`, json.b.b.`_1`.:Int64, json.b.b.`_1`.:UUID, json.b.b.`_2`, json.b.b.`_2`.:Int64, json.b.b.`_2`.:UUID, json.b.b.`_3`, json.b.b.`_3`.:Int64, json.b.b.`_3`.:UUID, json.b.b.`_4`, json.b.b.`_4`.:Int64, json.b.b.`_4`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test format Null;
|
||||
select json, json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.`_0`, json.b.b.`_0`.:Int64, json.b.b.`_0`.:UUID, json.b.b.`_1`, json.b.b.`_1`.:Int64, json.b.b.`_1`.:UUID, json.b.b.`_2`, json.b.b.`_2`.:Int64, json.b.b.`_2`.:UUID, json.b.b.`_3`, json.b.b.`_3`.:Int64, json.b.b.`_3`.:UUID, json.b.b.`_4`, json.b.b.`_4`.:Int64, json.b.b.`_4`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test order by id format Null;
|
||||
|
||||
select count() from test where json.non.existing.path is Null;
|
||||
select count() from test where json.non.existing.path.:String is Null;
|
||||
select json.non.existing.path from test order by id format Null;
|
||||
select json.non.existing.path.:Int64 from test order by id format Null;
|
||||
select json.non.existing.path, json.non.existing.path.:Int64 from test order by id format Null;
|
||||
select json, json.non.existing.path from test order by id format Null;
|
||||
select json, json.non.existing.path.:Int64 from test order by id format Null;
|
||||
select json, json.non.existing.path, json.non.existing.path.:Int64 from test format Null;
|
||||
select json, json.non.existing.path, json.non.existing.path.:Int64 from test order by id format Null;
|
||||
|
||||
select count() from test where json.a.b.c == 0;
|
||||
select json.a.b.c from test format Null;
|
||||
select json.a.b.c from test order by id format Null;
|
||||
select json, json.a.b.c from test format Null;
|
||||
select json, json.a.b.c from test order by id format Null;
|
||||
|
||||
select count() from test where json.b.b.e is Null;
|
||||
select count() from test where json.b.b.e.:String is Null;
|
||||
select json.b.b.e from test format Null;
|
||||
select json.b.b.e from test order by id format Null;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date from test format Null;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test format Null;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null;
|
||||
select json, json.b.b.e from test format Null;
|
||||
select json, json.b.b.e from test order by id format Null;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date from test format Null;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null;
|
||||
|
||||
select count() from test where json.b.b.e is Null and json.a.b.d is Null ;
|
||||
select count() from test where json.b.b.e.:String is Null and json.a.b.d.:Int64 is Null;
|
||||
select json.b.b.e, json.a.b.d from test order by id format Null;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.a.b.d from test order by id format Null;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
|
||||
select count() from test where json.b.b.e is Null and json.d.a is Null;
|
||||
select count() from test where json.b.b.e.:String is Null and empty(json.d.a.:`Array(Nullable(Int64))`);
|
||||
select json.b.b.e, json.d.a from test order by id format Null;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test order by id format Null;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.d.a from test order by id format Null;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test order by id format Null;
|
||||
|
||||
select count() from test where json.b.b.e is Null and json.d.a is Null and json.d.b is Null;
|
||||
select count() from test where json.b.b.e.:String is Null and empty(json.d.a.:`Array(Nullable(Int64))`) and json.d.b.:Int64 is Null;
|
||||
select json.b.b.e, json.d.a, json.d.b from test order by id format Null;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.d.a, json.d.b from test order by id format Null;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
|
||||
select count() from test where json.d.a is Null and json.d.b is Null;
|
||||
select count() from test where empty(json.d.a.:`Array(Nullable(Int64))`) and json.d.b.:Int64 is Null;
|
||||
select json.d.a, json.d.b from test order by id format Null;
|
||||
select json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
select json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
select json, json.d.a, json.d.b from test order by id format Null;
|
||||
select json, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test format Null;
|
||||
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
|
||||
select count() from test where json.d.a is Null and json.b.b.`_1` is Null;
|
||||
select count() from test where empty(json.d.a.:`Array(Nullable(Int64))`) and json.b.b.`_1`.:Int64 is Null;
|
||||
select json.d.a, json.b.b.`_1` from test order by id format Null;
|
||||
select json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b.`_1`.:Date from test order by id format Null;
|
||||
select json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b, json.b.b.`_1`.:Date from test order by id format Null;
|
||||
select json, json.d.a, json.b.b.`_1` from test order by id format Null;
|
||||
select json, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b.`_1`.:Date from test order by id format Null;
|
||||
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b, json.b.b.`_1`.:Date from test format Null;
|
||||
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b, json.b.b.`_1`.:Date from test order by id format Null;
|
||||
|
||||
select count() from test where JSONEmpty(json.^a) and json.a.b.c == 0;
|
||||
select json.^a, json.a.b.c from test order by id format Null;
|
||||
select json, json.^a, json.a.b.c from test format Null;
|
||||
select json, json.^a, json.a.b.c from test order by id format Null;
|
||||
|
||||
select count() from test where JSONEmpty(json.^a) and json.a.b.d is Null;
|
||||
select json.^a, json.a.b.d from test order by id format Null;
|
||||
select json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
select json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
select json, json.^a, json.a.b.d from test order by id format Null;
|
||||
select json, json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
select json, json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test format Null;
|
||||
select json, json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
|
||||
drop table test;
|
@ -1,6 +1,3 @@
|
||||
No merges
|
||||
insert
|
||||
test
|
||||
('a.b.c','UInt32')
|
||||
('a.b.d','Array(Nullable(String))')
|
||||
('a.b.d','DateTime64(9)')
|
||||
@ -34,8 +31,6 @@ test
|
||||
680000
|
||||
0
|
||||
0
|
||||
With merges
|
||||
test
|
||||
('a.b.c','UInt32')
|
||||
('a.b.d','Array(Nullable(String))')
|
||||
('a.b.d','DateTime64(9)')
|
@ -1,142 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest, long, no-debug, no-tsan, no-asan, no-msan, no-ubsan
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# reset --log_comment
|
||||
CLICKHOUSE_LOG_COMMENT=
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_json_type=1 --allow_experimental_variant_type=1 --use_variant_as_common_type=1 --session_timezone=UTC"
|
||||
|
||||
function insert()
|
||||
{
|
||||
echo "insert"
|
||||
$CH_CLIENT -q "truncate table test"
|
||||
$CH_CLIENT -q "insert into test select number, '{}' from numbers(100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number)) from numbers(100000, 100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number))) from numbers(200000, 100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('b.b.d', number::UInt32, 'b.b.e', 'str_' || toString(number))) from numbers(300000, 100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number))) from numbers(400000, 100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number), 'b.b._' || toString(number % 5), number::UInt32)) from numbers(500000, 100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', range(number % + 1)::Array(UInt32), 'a.b.e', 'str_' || toString(number), 'd.a', number::UInt32, 'd.c', toDate(number))) from numbers(600000, 100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', toDateTime(number), 'a.b.e', 'str_' || toString(number), 'd.a', range(number % 5 + 1)::Array(UInt32), 'd.b', number::UInt32)) from numbers(700000, 100000)"
|
||||
}
|
||||
|
||||
function test()
|
||||
{
|
||||
echo "test"
|
||||
$CH_CLIENT -q "select distinct arrayJoin(JSONAllPathsWithTypes(json)) as paths_with_types from test order by paths_with_types"
|
||||
|
||||
$CH_CLIENT -q "select json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.\`_0\`, json.b.b.\`_0\`.:Int64, json.b.b.\`_0\`.:UUID, json.b.b.\`_1\`, json.b.b.\`_1\`.:Int64, json.b.b.\`_1\`.:UUID, json.b.b.\`_2\`, json.b.b.\`_2\`.:Int64, json.b.b.\`_2\`.:UUID, json.b.b.\`_3\`, json.b.b.\`_3\`.:Int64, json.b.b.\`_3\`.:UUID, json.b.b.\`_4\`, json.b.b.\`_4\`.:Int64, json.b.b.\`_4\`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test format Null"
|
||||
$CH_CLIENT -q "select json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.\`_0\`, json.b.b.\`_0\`.:Int64, json.b.b.\`_0\`.:UUID, json.b.b.\`_1\`, json.b.b.\`_1\`.:Int64, json.b.b.\`_1\`.:UUID, json.b.b.\`_2\`, json.b.b.\`_2\`.:Int64, json.b.b.\`_2\`.:UUID, json.b.b.\`_3\`, json.b.b.\`_3\`.:Int64, json.b.b.\`_3\`.:UUID, json.b.b.\`_4\`, json.b.b.\`_4\`.:Int64, json.b.b.\`_4\`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.\`_0\`, json.b.b.\`_0\`.:Int64, json.b.b.\`_0\`.:UUID, json.b.b.\`_1\`, json.b.b.\`_1\`.:Int64, json.b.b.\`_1\`.:UUID, json.b.b.\`_2\`, json.b.b.\`_2\`.:Int64, json.b.b.\`_2\`.:UUID, json.b.b.\`_3\`, json.b.b.\`_3\`.:Int64, json.b.b.\`_3\`.:UUID, json.b.b.\`_4\`, json.b.b.\`_4\`.:Int64, json.b.b.\`_4\`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test format Null"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.\`_0\`, json.b.b.\`_0\`.:Int64, json.b.b.\`_0\`.:UUID, json.b.b.\`_1\`, json.b.b.\`_1\`.:Int64, json.b.b.\`_1\`.:UUID, json.b.b.\`_2\`, json.b.b.\`_2\`.:Int64, json.b.b.\`_2\`.:UUID, json.b.b.\`_3\`, json.b.b.\`_3\`.:Int64, json.b.b.\`_3\`.:UUID, json.b.b.\`_4\`, json.b.b.\`_4\`.:Int64, json.b.b.\`_4\`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.non.existing.path is Null"
|
||||
$CH_CLIENT -q "select count() from test where json.non.existing.path.:String is Null"
|
||||
$CH_CLIENT -q "select json.non.existing.path from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.non.existing.path.:Int64 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.non.existing.path, json.non.existing.path.:Int64 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path.:Int64 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path, json.non.existing.path.:Int64 from test format Null"
|
||||
$CH_CLIENT -q "select json, json.non.existing.path, json.non.existing.path.:Int64 from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.a.b.c == 0"
|
||||
$CH_CLIENT -q "select json.a.b.c from test format Null"
|
||||
$CH_CLIENT -q "select json.a.b.c from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.a.b.c from test format Null"
|
||||
$CH_CLIENT -q "select json, json.a.b.c from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e is Null"
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e.:String is Null"
|
||||
$CH_CLIENT -q "select json.b.b.e from test format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e from test format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e is Null and json.a.b.d is Null "
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e.:String is Null and json.a.b.d.:Int64 is Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.a.b.d from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.a.b.d from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e is Null and json.d.a is Null"
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e.:String is Null and empty(json.d.a.:\`Array(Nullable(Int64))\`)"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.d.a from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.d.a from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e is Null and json.d.a is Null and json.d.b is Null"
|
||||
$CH_CLIENT -q "select count() from test where json.b.b.e.:String is Null and empty(json.d.a.:\`Array(Nullable(Int64))\`) and json.d.b.:Int64 is Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.d.a, json.d.b from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.d.a, json.d.b from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.d.a is Null and json.d.b is Null"
|
||||
$CH_CLIENT -q "select count() from test where empty(json.d.a.:\`Array(Nullable(Int64))\`) and json.d.b.:Int64 is Null"
|
||||
$CH_CLIENT -q "select json.d.a, json.d.b from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.b from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where json.d.a is Null and json.b.b.\`_1\` is Null"
|
||||
$CH_CLIENT -q "select count() from test where empty(json.d.a.:\`Array(Nullable(Int64))\`) and json.b.b.\`_1\`.:Int64 is Null"
|
||||
$CH_CLIENT -q "select json.d.a, json.b.b.\`_1\` from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_1\`.:Int64, json.b.b.\`_1\`.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_1\`.:Int64, json.b.b, json.b.b.\`_1\`.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.b.b.\`_1\` from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_1\`.:Int64, json.b.b.\`_1\`.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_1\`.:Int64, json.b.b, json.b.b.\`_1\`.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.d.a, json.d.a.:\`Array(Nullable(Int64))\`, json.d.a.:Date, json.b.b.\`_1\`.:Int64, json.b.b, json.b.b.\`_1\`.:Date from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where JSONEmpty(json.^a) and json.a.b.c == 0"
|
||||
$CH_CLIENT -q "select json.^a, json.a.b.c from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.c from test format Null"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.c from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where JSONEmpty(json.^a) and json.a.b.d is Null"
|
||||
$CH_CLIENT -q "select json.^a, json.a.b.d from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.d from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test format Null"
|
||||
$CH_CLIENT -q "select json, json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null"
|
||||
}
|
||||
|
||||
$CH_CLIENT -q "drop table if exists test;"
|
||||
|
||||
$CH_CLIENT -q "create table test (id UInt64, json JSON(max_dynamic_paths=2, a.b.c UInt32)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;"
|
||||
echo "No merges"
|
||||
$CH_CLIENT -q "system stop merges test"
|
||||
insert
|
||||
test
|
||||
echo "With merges"
|
||||
$CH_CLIENT -q "system start merges test"
|
||||
test
|
||||
$CH_CLIENT -q "drop table test;"
|
@ -0,0 +1,128 @@
|
||||
-- Tags: no-fasttest, long, no-debug, no-tsan, no-asan, no-msan, no-ubsan
|
||||
|
||||
set allow_experimental_json_type = 1;
|
||||
set allow_experimental_variant_type = 1;
|
||||
set use_variant_as_common_type = 1;
|
||||
set session_timezone = 'UTC';
|
||||
|
||||
drop table if exists test;
|
||||
create table test (id UInt64, json JSON(max_dynamic_paths=2, a.b.c UInt32)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;
|
||||
|
||||
insert into test select number, '{}' from numbers(100000);
|
||||
insert into test select number, toJSONString(map('a.b.c', number)) from numbers(100000, 100000);
|
||||
insert into test select number, toJSONString(map('a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number))) from numbers(200000, 100000);
|
||||
insert into test select number, toJSONString(map('b.b.d', number::UInt32, 'b.b.e', 'str_' || toString(number))) from numbers(300000, 100000);
|
||||
insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number))) from numbers(400000, 100000);
|
||||
insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', number::UInt32, 'a.b.e', 'str_' || toString(number), 'b.b._' || toString(number % 5), number::UInt32)) from numbers(500000, 100000);
|
||||
insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', range(number % + 1)::Array(UInt32), 'a.b.e', 'str_' || toString(number), 'd.a', number::UInt32, 'd.c', toDate(number))) from numbers(600000, 100000);
|
||||
insert into test select number, toJSONString(map('a.b.c', number, 'a.b.d', toDateTime(number), 'a.b.e', 'str_' || toString(number), 'd.a', range(number % 5 + 1)::Array(UInt32), 'd.b', number::UInt32)) from numbers(700000, 100000);
|
||||
|
||||
{% for merge_command in ['system stop merges test', 'system start merges test'] -%}
|
||||
|
||||
{{ merge_command }};
|
||||
|
||||
select distinct arrayJoin(JSONAllPathsWithTypes(json)) as paths_with_types from test order by paths_with_types;
|
||||
|
||||
select json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.`_0`, json.b.b.`_0`.:Int64, json.b.b.`_0`.:UUID, json.b.b.`_1`, json.b.b.`_1`.:Int64, json.b.b.`_1`.:UUID, json.b.b.`_2`, json.b.b.`_2`.:Int64, json.b.b.`_2`.:UUID, json.b.b.`_3`, json.b.b.`_3`.:Int64, json.b.b.`_3`.:UUID, json.b.b.`_4`, json.b.b.`_4`.:Int64, json.b.b.`_4`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test format Null;
|
||||
select json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.`_0`, json.b.b.`_0`.:Int64, json.b.b.`_0`.:UUID, json.b.b.`_1`, json.b.b.`_1`.:Int64, json.b.b.`_1`.:UUID, json.b.b.`_2`, json.b.b.`_2`.:Int64, json.b.b.`_2`.:UUID, json.b.b.`_3`, json.b.b.`_3`.:Int64, json.b.b.`_3`.:UUID, json.b.b.`_4`, json.b.b.`_4`.:Int64, json.b.b.`_4`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test order by id format Null;
|
||||
select json, json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.`_0`, json.b.b.`_0`.:Int64, json.b.b.`_0`.:UUID, json.b.b.`_1`, json.b.b.`_1`.:Int64, json.b.b.`_1`.:UUID, json.b.b.`_2`, json.b.b.`_2`.:Int64, json.b.b.`_2`.:UUID, json.b.b.`_3`, json.b.b.`_3`.:Int64, json.b.b.`_3`.:UUID, json.b.b.`_4`, json.b.b.`_4`.:Int64, json.b.b.`_4`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test format Null;
|
||||
select json, json.non.existing.path, json.a.b.c, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:UUID, json.a.b.e, json.a.b.e.:String, json.a.b.e.:UUID, json.b.b.`_0`, json.b.b.`_0`.:Int64, json.b.b.`_0`.:UUID, json.b.b.`_1`, json.b.b.`_1`.:Int64, json.b.b.`_1`.:UUID, json.b.b.`_2`, json.b.b.`_2`.:Int64, json.b.b.`_2`.:UUID, json.b.b.`_3`, json.b.b.`_3`.:Int64, json.b.b.`_3`.:UUID, json.b.b.`_4`, json.b.b.`_4`.:Int64, json.b.b.`_4`.:UUID, json.b.b.d, json.b.b.d.:Int64, json.b.b.d.:UUID, json.b.b.e, json.b.b.e.:String, json.b.b.e.:UUID, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:UUID, json.d.b, json.d.b.:Int64, json.d.b.:UUID, json.d.c, json.d.c.:Date, json.d.c.:UUID, json.^n, json.^a, json.^a.b, json.^b, json.^d from test order by id format Null;
|
||||
|
||||
select count() from test where json.non.existing.path is Null;
|
||||
select count() from test where json.non.existing.path.:String is Null;
|
||||
select json.non.existing.path from test order by id format Null;
|
||||
select json.non.existing.path.:Int64 from test order by id format Null;
|
||||
select json.non.existing.path, json.non.existing.path.:Int64 from test order by id format Null;
|
||||
select json, json.non.existing.path from test order by id format Null;
|
||||
select json, json.non.existing.path.:Int64 from test order by id format Null;
|
||||
select json, json.non.existing.path, json.non.existing.path.:Int64 from test format Null;
|
||||
select json, json.non.existing.path, json.non.existing.path.:Int64 from test order by id format Null;
|
||||
|
||||
select count() from test where json.a.b.c == 0;
|
||||
select json.a.b.c from test format Null;
|
||||
select json.a.b.c from test order by id format Null;
|
||||
select json, json.a.b.c from test format Null;
|
||||
select json, json.a.b.c from test order by id format Null;
|
||||
|
||||
select count() from test where json.b.b.e is Null;
|
||||
select count() from test where json.b.b.e.:String is Null;
|
||||
select json.b.b.e from test format Null;
|
||||
select json.b.b.e from test order by id format Null;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date from test format Null;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test format Null;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null;
|
||||
select json, json.b.b.e from test format Null;
|
||||
select json, json.b.b.e from test order by id format Null;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date from test format Null;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date from test order by id format Null;
|
||||
|
||||
select count() from test where json.b.b.e is Null and json.a.b.d is Null ;
|
||||
select count() from test where json.b.b.e.:String is Null and json.a.b.d.:Int64 is Null;
|
||||
select json.b.b.e, json.a.b.d from test order by id format Null;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.a.b.d from test order by id format Null;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
|
||||
select count() from test where json.b.b.e is Null and json.d.a is Null;
|
||||
select count() from test where json.b.b.e.:String is Null and empty(json.d.a.:`Array(Nullable(Int64))`);
|
||||
select json.b.b.e, json.d.a from test order by id format Null;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test order by id format Null;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.d.a from test order by id format Null;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date from test order by id format Null;
|
||||
|
||||
select count() from test where json.b.b.e is Null and json.d.a is Null and json.d.b is Null;
|
||||
select count() from test where json.b.b.e.:String is Null and empty(json.d.a.:`Array(Nullable(Int64))`) and json.d.b.:Int64 is Null;
|
||||
select json.b.b.e, json.d.a, json.d.b from test order by id format Null;
|
||||
select json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
select json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.d.a, json.d.b from test order by id format Null;
|
||||
select json, json.b.b.e.:String, json.b.b.e.:Date, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test format Null;
|
||||
select json, json.b.b.e, json.b.b.e.:String, json.b.b.e.:Date, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
|
||||
select count() from test where json.d.a is Null and json.d.b is Null;
|
||||
select count() from test where empty(json.d.a.:`Array(Nullable(Int64))`) and json.d.b.:Int64 is Null;
|
||||
select json.d.a, json.d.b from test order by id format Null;
|
||||
select json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
select json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
select json, json.d.a, json.d.b from test order by id format Null;
|
||||
select json, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test format Null;
|
||||
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.d.b, json.d.b.:Int64, json.d.b.:Date from test order by id format Null;
|
||||
|
||||
select count() from test where json.d.a is Null and json.b.b.`_1` is Null;
|
||||
select count() from test where empty(json.d.a.:`Array(Nullable(Int64))`) and json.b.b.`_1`.:Int64 is Null;
|
||||
select json.d.a, json.b.b.`_1` from test order by id format Null;
|
||||
select json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b.`_1`.:Date from test order by id format Null;
|
||||
select json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b, json.b.b.`_1`.:Date from test order by id format Null;
|
||||
select json, json.d.a, json.b.b.`_1` from test order by id format Null;
|
||||
select json, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b.`_1`.:Date from test order by id format Null;
|
||||
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b, json.b.b.`_1`.:Date from test format Null;
|
||||
select json, json.d.a, json.d.a.:`Array(Nullable(Int64))`, json.d.a.:Date, json.b.b.`_1`.:Int64, json.b.b, json.b.b.`_1`.:Date from test order by id format Null;
|
||||
|
||||
select count() from test where JSONEmpty(json.^a) and json.a.b.c == 0;
|
||||
select json.^a, json.a.b.c from test order by id format Null;
|
||||
select json, json.^a, json.a.b.c from test format Null;
|
||||
select json, json.^a, json.a.b.c from test order by id format Null;
|
||||
|
||||
select count() from test where JSONEmpty(json.^a) and json.a.b.d is Null;
|
||||
select json.^a, json.a.b.d from test order by id format Null;
|
||||
select json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
select json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
select json, json.^a, json.a.b.d from test order by id format Null;
|
||||
select json, json.^a, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
select json, json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test format Null;
|
||||
select json, json.^a, json.a.b.d, json.a.b.d.:Int64, json.a.b.d.:Date from test order by id format Null;
|
||||
|
||||
{% endfor -%}
|
||||
|
||||
drop table test;
|
@ -1,6 +1,3 @@
|
||||
Memory
|
||||
insert
|
||||
test
|
||||
('a.a1','String')
|
||||
('a.a2','String')
|
||||
('a.a3','String')
|
||||
@ -110,10 +107,6 @@ test
|
||||
"json.a.r.:`Array(JSON)`.^`b`.c": [[], [], [], [], [], [], [], [], [], [], [{"d_0":"10"}], [{"d_0":"11"},{"d_1":"11"}], [{"d_0":"12"},{"d_1":"12"},{"d_2":"12"}], [{"d_0":"13"},{"d_1":"13"},{"d_2":"13"},{"d_3":"13"}], [{"d_0":"14"},{"d_1":"14"},{"d_2":"14"},{"d_3":"14"},{"d_4":"14"}], [{"d_0":"15"}], [{"d_0":"16"},{"d_1":"16"}], [{"d_0":"17"},{"d_1":"17"},{"d_2":"17"}], [{"d_0":"18"},{"d_1":"18"},{"d_2":"18"},{"d_3":"18"}], [{"d_0":"19"},{"d_1":"19"},{"d_2":"19"},{"d_3":"19"},{"d_4":"19"}]],
|
||||
"json.a.r.:`Array(JSON)`.b.c.d_0.:`Int64`": [[], [], [], [], [], [], [], [], [], [], ["10"], ["11",null], ["12",null,null], ["13",null,null,null], ["14",null,null,null,null], ["15"], ["16",null], ["17",null,null], ["18",null,null,null], ["19",null,null,null,null]]
|
||||
}
|
||||
MergeTree compact
|
||||
No merges
|
||||
insert
|
||||
test
|
||||
('a.a1','String')
|
||||
('a.a2','String')
|
||||
('a.a3','String')
|
||||
@ -223,8 +216,6 @@ test
|
||||
"json.a.r.:`Array(JSON)`.^`b`.c": [[], [], [], [], [], [], [], [], [], [], [{"d_0":"10"}], [{"d_0":"11"},{"d_1":"11"}], [{"d_0":"12"},{"d_1":"12"},{"d_2":"12"}], [{"d_0":"13"},{"d_1":"13"},{"d_2":"13"},{"d_3":"13"}], [{"d_0":"14"},{"d_1":"14"},{"d_2":"14"},{"d_3":"14"},{"d_4":"14"}], [{"d_0":"15"}], [{"d_0":"16"},{"d_1":"16"}], [{"d_0":"17"},{"d_1":"17"},{"d_2":"17"}], [{"d_0":"18"},{"d_1":"18"},{"d_2":"18"},{"d_3":"18"}], [{"d_0":"19"},{"d_1":"19"},{"d_2":"19"},{"d_3":"19"},{"d_4":"19"}]],
|
||||
"json.a.r.:`Array(JSON)`.b.c.d_0.:`Int64`": [[], [], [], [], [], [], [], [], [], [], ["10"], ["11",null], ["12",null,null], ["13",null,null,null], ["14",null,null,null,null], ["15"], ["16",null], ["17",null,null], ["18",null,null,null], ["19",null,null,null,null]]
|
||||
}
|
||||
With merges
|
||||
test
|
||||
('a.a1','String')
|
||||
('a.a2','String')
|
||||
('a.a3','String')
|
||||
@ -334,10 +325,6 @@ test
|
||||
"json.a.r.:`Array(JSON)`.^`b`.c": [[], [], [], [], [], [], [], [], [], [], [{"d_0":"10"}], [{"d_0":"11"},{"d_1":"11"}], [{"d_0":"12"},{"d_1":"12"},{"d_2":"12"}], [{"d_0":"13"},{"d_1":"13"},{"d_2":"13"},{"d_3":"13"}], [{"d_0":"14"},{"d_1":"14"},{"d_2":"14"},{"d_3":"14"},{"d_4":"14"}], [{"d_0":"15"}], [{"d_0":"16"},{"d_1":"16"}], [{"d_0":"17"},{"d_1":"17"},{"d_2":"17"}], [{"d_0":"18"},{"d_1":"18"},{"d_2":"18"},{"d_3":"18"}], [{"d_0":"19"},{"d_1":"19"},{"d_2":"19"},{"d_3":"19"},{"d_4":"19"}]],
|
||||
"json.a.r.:`Array(JSON)`.b.c.d_0.:`Int64`": [[], [], [], [], [], [], [], [], [], [], ["10"], ["11",null], ["12",null,null], ["13",null,null,null], ["14",null,null,null,null], ["15"], ["16",null], ["17",null,null], ["18",null,null,null], ["19",null,null,null,null]]
|
||||
}
|
||||
MergeTree wide
|
||||
No merges
|
||||
insert
|
||||
test
|
||||
('a.a1','String')
|
||||
('a.a2','String')
|
||||
('a.a3','String')
|
||||
@ -447,8 +434,6 @@ test
|
||||
"json.a.r.:`Array(JSON)`.^`b`.c": [[], [], [], [], [], [], [], [], [], [], [{"d_0":"10"}], [{"d_0":"11"},{"d_1":"11"}], [{"d_0":"12"},{"d_1":"12"},{"d_2":"12"}], [{"d_0":"13"},{"d_1":"13"},{"d_2":"13"},{"d_3":"13"}], [{"d_0":"14"},{"d_1":"14"},{"d_2":"14"},{"d_3":"14"},{"d_4":"14"}], [{"d_0":"15"}], [{"d_0":"16"},{"d_1":"16"}], [{"d_0":"17"},{"d_1":"17"},{"d_2":"17"}], [{"d_0":"18"},{"d_1":"18"},{"d_2":"18"},{"d_3":"18"}], [{"d_0":"19"},{"d_1":"19"},{"d_2":"19"},{"d_3":"19"},{"d_4":"19"}]],
|
||||
"json.a.r.:`Array(JSON)`.b.c.d_0.:`Int64`": [[], [], [], [], [], [], [], [], [], [], ["10"], ["11",null], ["12",null,null], ["13",null,null,null], ["14",null,null,null,null], ["15"], ["16",null], ["17",null,null], ["18",null,null,null], ["19",null,null,null,null]]
|
||||
}
|
||||
With merges
|
||||
test
|
||||
('a.a1','String')
|
||||
('a.a2','String')
|
||||
('a.a3','String')
|
@ -1,71 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest, long
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# reset --log_comment
|
||||
CLICKHOUSE_LOG_COMMENT=
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_json_type=1 --allow_experimental_variant_type=1 --use_variant_as_common_type=1"
|
||||
|
||||
function insert()
|
||||
{
|
||||
echo "insert"
|
||||
$CH_CLIENT -q "truncate table test"
|
||||
$CH_CLIENT -q "insert into test select number, '{}' from numbers(5)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(5, 5)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.r', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(10, 5)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.a1', number, 'a.a2', number, 'a.a3', number, 'a.a4', number, 'a.a5', number, 'a.a6', number, 'a.a7', number, 'a.a8', number, 'a.r', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(15, 5)"
|
||||
}
|
||||
|
||||
function test()
|
||||
{
|
||||
echo "test"
|
||||
$CH_CLIENT -q "select distinct arrayJoin(JSONAllPathsWithTypes(json)) as paths_with_types from test order by paths_with_types"
|
||||
$CH_CLIENT -q "select distinct arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.b))) as paths_with_types from test order by paths_with_types"
|
||||
$CH_CLIENT -q "select distinct arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.r[]))) as paths_with_types from test order by paths_with_types"
|
||||
|
||||
$CH_CLIENT -q "select json, json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.a.r[].c.d.e.:\`Array(Nullable(Int64))\`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].c.d.e.:\`Array(Nullable(Int64))\`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test order by id format JSONColumns"
|
||||
|
||||
$CH_CLIENT -q "select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test order by id format JSONColumns"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test order by id format JSONColumns"
|
||||
}
|
||||
|
||||
$CH_CLIENT -q "drop table if exists test;"
|
||||
|
||||
echo "Memory"
|
||||
$CH_CLIENT -q "create table test (id UInt64, json JSON(max_dynamic_paths=8, a.b Array(JSON))) engine=Memory"
|
||||
insert
|
||||
test
|
||||
$CH_CLIENT -q "drop table test;"
|
||||
|
||||
echo "MergeTree compact"
|
||||
$CH_CLIENT -q "create table test (id UInt64, json JSON(max_dynamic_paths=8, a.b Array(JSON))) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;"
|
||||
echo "No merges"
|
||||
$CH_CLIENT -q "system stop merges test"
|
||||
insert
|
||||
test
|
||||
echo "With merges"
|
||||
$CH_CLIENT -q "system start merges test"
|
||||
test
|
||||
$CH_CLIENT -q "drop table test;"
|
||||
|
||||
echo "MergeTree wide"
|
||||
$CH_CLIENT -q "create table test (id UInt64, json JSON(max_dynamic_paths=8, a.b Array(JSON))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;"
|
||||
echo "No merges"
|
||||
$CH_CLIENT -q "system stop merges test"
|
||||
insert
|
||||
test
|
||||
echo "With merges"
|
||||
$CH_CLIENT -q "system start merges test"
|
||||
test
|
||||
$CH_CLIENT -q "drop table test;"
|
@ -0,0 +1,41 @@
|
||||
-- Tags: no-fasttest, long
|
||||
|
||||
set allow_experimental_json_type = 1;
|
||||
set allow_experimental_variant_type = 1;
|
||||
set use_variant_as_common_type = 1;
|
||||
|
||||
drop table if exists test;
|
||||
|
||||
{% for create_command in ['create table test (id UInt64, json JSON(max_dynamic_paths=8, a.b Array(JSON))) engine=Memory;',
|
||||
'create table test (id UInt64, json JSON(max_dynamic_paths=8, a.b Array(JSON))) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000; system stop merges test;',
|
||||
'create table test (id UInt64, json JSON(max_dynamic_paths=8, a.b Array(JSON))) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000; system start merges test;',
|
||||
'create table test (id UInt64, json JSON(max_dynamic_paths=8, a.b Array(JSON))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1; system stop merges test;',
|
||||
'create table test (id UInt64, json JSON(max_dynamic_paths=8, a.b Array(JSON))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1; system start merges test;'] -%}
|
||||
|
||||
{{ create_command }}
|
||||
|
||||
insert into test select number, '{}' from numbers(5);
|
||||
insert into test select number, toJSONString(map('a.b', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(5, 5);
|
||||
insert into test select number, toJSONString(map('a.r', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(10, 5);
|
||||
insert into test select number, toJSONString(map('a.a1', number, 'a.a2', number, 'a.a3', number, 'a.a4', number, 'a.a5', number, 'a.a6', number, 'a.a7', number, 'a.a8', number, 'a.r', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(15, 5);
|
||||
|
||||
select distinct arrayJoin(JSONAllPathsWithTypes(json)) as paths_with_types from test order by paths_with_types;
|
||||
select distinct arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.b))) as paths_with_types from test order by paths_with_types;
|
||||
select distinct arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.r[]))) as paths_with_types from test order by paths_with_types;
|
||||
|
||||
select json, json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test order by id format JSONColumns;
|
||||
select json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test order by id format JSONColumns;
|
||||
|
||||
select json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test order by id format JSONColumns;
|
||||
select json.a.r[].c.d.e.:`Array(Nullable(Int64))`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test order by id format JSONColumns;
|
||||
select json.a.r, json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test order by id format JSONColumns;
|
||||
select json.a.r, json.a.r[].c.d.e.:`Array(Nullable(Int64))`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test order by id format JSONColumns;
|
||||
|
||||
select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test order by id format JSONColumns;
|
||||
select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test order by id format JSONColumns;
|
||||
select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test order by id format JSONColumns;
|
||||
select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test order by id format JSONColumns;
|
||||
|
||||
drop table test;
|
||||
|
||||
{% endfor -%}
|
@ -1,6 +1,3 @@
|
||||
No merges
|
||||
insert
|
||||
test
|
||||
('a.a1','String')
|
||||
('a.a2','String')
|
||||
('a.a3','String')
|
||||
@ -31,8 +28,6 @@ test
|
||||
20000
|
||||
0
|
||||
0
|
||||
With merges
|
||||
test
|
||||
('a.a1','String')
|
||||
('a.a2','String')
|
||||
('a.a3','String')
|
@ -1,73 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest, long, no-debug, no-tsan, no-asan, no-msan, no-ubsan
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# reset --log_comment
|
||||
CLICKHOUSE_LOG_COMMENT=
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_json_type=1 --allow_experimental_variant_type=1 --use_variant_as_common_type=1"
|
||||
|
||||
function insert()
|
||||
{
|
||||
echo "insert"
|
||||
$CH_CLIENT -q "truncate table test"
|
||||
$CH_CLIENT -q "insert into test select number, '{}' from numbers(10000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(10000, 10000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.r', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(20000, 10000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.a1', number, 'a.a2', number, 'a.a3', number, 'a.a4', number, 'a.a5', number, 'a.a6', number, 'a.a7', number, 'a.a8', number, 'a.r', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(30000, 10000)"
|
||||
}
|
||||
|
||||
function test()
|
||||
{
|
||||
echo "test"
|
||||
$CH_CLIENT -q "select distinct arrayJoin(JSONAllPathsWithTypes(json)) as paths_with_types from test order by paths_with_types"
|
||||
$CH_CLIENT -q "select distinct arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.b))) as paths_with_types from test order by paths_with_types"
|
||||
$CH_CLIENT -q "select distinct arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.r[]))) as paths_with_types from test order by paths_with_types"
|
||||
|
||||
$CH_CLIENT -q "select json, json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test format Null"
|
||||
$CH_CLIENT -q "select json, json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test format Null"
|
||||
$CH_CLIENT -q "select json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where empty(json.a.r[].c.d.e) and empty(json.a.r[].b.c.d_0) and empty(json.a.r[].b.c.d_1)"
|
||||
$CH_CLIENT -q "select count() from test where empty(json.a.r[].c.d.e.:\`Array(Nullable(Int64))\`) and empty(json.a.r[].b.c.d_0.:Int64) and empty(json.a.r[].b.c.d_1.:Int64)"
|
||||
$CH_CLIENT -q "select count() from test where arrayJoin(json.a.r[].c.d.e) is null and arrayJoin(json.a.r[].b.c.d_0) is null and arrayJoin(json.a.r[].b.c.d_1) is null"
|
||||
$CH_CLIENT -q "select count() from test where arrayJoin(json.a.r[].c.d.e.:\`Array(Nullable(Int64))\`) is null and arrayJoin(json.a.r[].b.c.d_0.:Int64) is null and arrayJoin(json.a.r[].b.c.d_1.:Int64) is null"
|
||||
|
||||
$CH_CLIENT -q "select json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.r[].c.d.e.:\`Array(Nullable(Int64))\`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r[].c.d.e.:\`Array(Nullable(Int64))\`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].c.d.e.:\`Array(Nullable(Int64))\`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].c.d.e.:\`Array(Nullable(Int64))\`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where empty(json.a.r[].^b) and empty(json.a.r[].^b.c) and empty(json.a.r[].b.c.d_0)"
|
||||
$CH_CLIENT -q "select count() from test where empty(json.a.r[].^b) and empty(json.a.r[].^b.c) and empty(json.a.r[].b.c.d_0.:Int64)"
|
||||
$CH_CLIENT -q "select count() from test where JSONEmpty(arrayJoin(json.a.r[].^b)) and JSONEmpty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0) is null"
|
||||
$CH_CLIENT -q "select count() from test where JSONEmpty(arrayJoin(json.a.r[].^b)) and JSONEmpty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0.:Int64) is null"
|
||||
|
||||
$CH_CLIENT -q "select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test order by id format Null"
|
||||
}
|
||||
|
||||
$CH_CLIENT -q "drop table if exists test;"
|
||||
|
||||
$CH_CLIENT -q "create table test (id UInt64, json JSON(max_dynamic_paths=8, a.b Array(JSON))) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;"
|
||||
echo "No merges"
|
||||
$CH_CLIENT -q "system stop merges test"
|
||||
insert
|
||||
test
|
||||
echo "With merges"
|
||||
$CH_CLIENT -q "system start merges test"
|
||||
test
|
||||
$CH_CLIENT -q "drop table test;"
|
@ -0,0 +1,57 @@
|
||||
-- Tags: no-fasttest, long, no-debug, no-tsan, no-asan, no-msan, no-ubsan
|
||||
|
||||
set allow_experimental_json_type = 1;
|
||||
set allow_experimental_variant_type = 1;
|
||||
set use_variant_as_common_type = 1;
|
||||
|
||||
create table test (id UInt64, json JSON(max_dynamic_paths=8, a.b Array(JSON))) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;
|
||||
|
||||
insert into test select number, '{}' from numbers(10000);
|
||||
insert into test select number, toJSONString(map('a.b', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(10000, 10000);
|
||||
insert into test select number, toJSONString(map('a.r', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(20000, 10000);
|
||||
insert into test select number, toJSONString(map('a.a1', number, 'a.a2', number, 'a.a3', number, 'a.a4', number, 'a.a5', number, 'a.a6', number, 'a.a7', number, 'a.a8', number, 'a.r', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(30000, 10000);
|
||||
|
||||
{% for merge_command in ['system stop merges test', 'system start merges test'] -%}
|
||||
|
||||
{{ merge_command }};
|
||||
|
||||
select distinct arrayJoin(JSONAllPathsWithTypes(json)) as paths_with_types from test order by paths_with_types;
|
||||
select distinct arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.b))) as paths_with_types from test order by paths_with_types;
|
||||
select distinct arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.r[]))) as paths_with_types from test order by paths_with_types;
|
||||
|
||||
select json, json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test format Null;
|
||||
select json, json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test order by id format Null;
|
||||
select json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test format Null;
|
||||
select json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test order by id format Null;
|
||||
|
||||
select count() from test where empty(json.a.r[].c.d.e) and empty(json.a.r[].b.c.d_0) and empty(json.a.r[].b.c.d_1);
|
||||
select count() from test where empty(json.a.r[].c.d.e.:`Array(Nullable(Int64))`) and empty(json.a.r[].b.c.d_0.:Int64) and empty(json.a.r[].b.c.d_1.:Int64);
|
||||
select count() from test where arrayJoin(json.a.r[].c.d.e) is null and arrayJoin(json.a.r[].b.c.d_0) is null and arrayJoin(json.a.r[].b.c.d_1) is null;
|
||||
select count() from test where arrayJoin(json.a.r[].c.d.e.:`Array(Nullable(Int64))`) is null and arrayJoin(json.a.r[].b.c.d_0.:Int64) is null and arrayJoin(json.a.r[].b.c.d_1.:Int64) is null;
|
||||
|
||||
select json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test format Null;
|
||||
select json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test order by id format Null;
|
||||
select json.a.r[].c.d.e.:`Array(Nullable(Int64))`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test format Null;
|
||||
select json.a.r[].c.d.e.:`Array(Nullable(Int64))`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test order by id format Null;
|
||||
select json.a.r, json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test format Null;
|
||||
select json.a.r, json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test order by id format Null;
|
||||
select json.a.r, json.a.r[].c.d.e.:`Array(Nullable(Int64))`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test format Null;
|
||||
select json.a.r, json.a.r[].c.d.e.:`Array(Nullable(Int64))`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test order by id format Null;
|
||||
|
||||
select count() from test where empty(json.a.r[].^b) and empty(json.a.r[].^b.c) and empty(json.a.r[].b.c.d_0);
|
||||
select count() from test where empty(json.a.r[].^b) and empty(json.a.r[].^b.c) and empty(json.a.r[].b.c.d_0.:Int64);
|
||||
select count() from test where JSONEmpty(arrayJoin(json.a.r[].^b)) and JSONEmpty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0) is null;
|
||||
select count() from test where JSONEmpty(arrayJoin(json.a.r[].^b)) and JSONEmpty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0.:Int64) is null;
|
||||
|
||||
select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test format Null;
|
||||
select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test order by id format Null;
|
||||
select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test format Null;
|
||||
select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test order by id format Null;
|
||||
select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test format Null;
|
||||
select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test order by id format Null;
|
||||
select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test format Null;
|
||||
select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test order by id format Null;
|
||||
|
||||
{% endfor -%}
|
||||
|
||||
drop table test;
|
@ -1,5 +1,3 @@
|
||||
insert
|
||||
test
|
||||
('a.a1','String')
|
||||
('a.a2','String')
|
||||
('a.a3','String')
|
||||
|
@ -1,68 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest, long, no-debug, no-tsan, no-asan, no-msan, no-ubsan
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# reset --log_comment
|
||||
CLICKHOUSE_LOG_COMMENT=
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_json_type=1 --allow_experimental_variant_type=1 --use_variant_as_common_type=1"
|
||||
|
||||
function insert()
|
||||
{
|
||||
echo "insert"
|
||||
$CH_CLIENT -q "truncate table test"
|
||||
$CH_CLIENT -q "insert into test select number, '{}' from numbers(10000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(10000, 10000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.r', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(20000, 10000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.a1', number, 'a.a2', number, 'a.a3', number, 'a.a4', number, 'a.a5', number, 'a.a6', number, 'a.a7', number, 'a.a8', number, 'a.r', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(30000, 10000)"
|
||||
}
|
||||
|
||||
function test()
|
||||
{
|
||||
echo "test"
|
||||
$CH_CLIENT -q "select distinct arrayJoin(JSONAllPathsWithTypes(json)) as paths_with_types from test order by paths_with_types"
|
||||
$CH_CLIENT -q "select distinct arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.b))) as paths_with_types from test order by paths_with_types"
|
||||
$CH_CLIENT -q "select distinct arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.r[]))) as paths_with_types from test order by paths_with_types"
|
||||
|
||||
$CH_CLIENT -q "select json, json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test format Null"
|
||||
$CH_CLIENT -q "select json, json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test format Null"
|
||||
$CH_CLIENT -q "select json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where empty(json.a.r[].c.d.e) and empty(json.a.r[].b.c.d_0) and empty(json.a.r[].b.c.d_1)"
|
||||
$CH_CLIENT -q "select count() from test where empty(json.a.r[].c.d.e.:\`Array(Nullable(Int64))\`) and empty(json.a.r[].b.c.d_0.:Int64) and empty(json.a.r[].b.c.d_1.:Int64)"
|
||||
$CH_CLIENT -q "select count() from test where arrayJoin(json.a.r[].c.d.e) is null and arrayJoin(json.a.r[].b.c.d_0) is null and arrayJoin(json.a.r[].b.c.d_1) is null"
|
||||
$CH_CLIENT -q "select count() from test where arrayJoin(json.a.r[].c.d.e.:\`Array(Nullable(Int64))\`) is null and arrayJoin(json.a.r[].b.c.d_0.:Int64) is null and arrayJoin(json.a.r[].b.c.d_1.:Int64) is null"
|
||||
|
||||
$CH_CLIENT -q "select json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.r[].c.d.e.:\`Array(Nullable(Int64))\`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r[].c.d.e.:\`Array(Nullable(Int64))\`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].c.d.e.:\`Array(Nullable(Int64))\`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].c.d.e.:\`Array(Nullable(Int64))\`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where empty(json.a.r[].^b) and empty(json.a.r[].^b.c) and empty(json.a.r[].b.c.d_0)"
|
||||
$CH_CLIENT -q "select count() from test where empty(json.a.r[].^b) and empty(json.a.r[].^b.c) and empty(json.a.r[].b.c.d_0.:Int64)"
|
||||
$CH_CLIENT -q "select count() from test where JSONEmpty(arrayJoin(json.a.r[].^b)) and JSONEmpty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0) is null"
|
||||
$CH_CLIENT -q "select count() from test where JSONEmpty(arrayJoin(json.a.r[].^b)) and JSONEmpty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0.:Int64) is null"
|
||||
|
||||
$CH_CLIENT -q "select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test order by id format Null"
|
||||
}
|
||||
|
||||
$CH_CLIENT -q "drop table if exists test;"
|
||||
|
||||
$CH_CLIENT -q "create table test (id UInt64, json JSON(max_dynamic_paths=8, a.b Array(JSON))) engine=Memory"
|
||||
insert
|
||||
test
|
||||
$CH_CLIENT -q "drop table test;"
|
@ -0,0 +1,52 @@
|
||||
-- Tags: no-fasttest, long, no-debug, no-tsan, no-asan, no-msan, no-ubsan
|
||||
|
||||
set allow_experimental_json_type = 1;
|
||||
set allow_experimental_variant_type = 1;
|
||||
set use_variant_as_common_type = 1;
|
||||
|
||||
drop table if exists test;
|
||||
create table test (id UInt64, json JSON(max_dynamic_paths=8, a.b Array(JSON))) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;
|
||||
|
||||
insert into test select number, '{}' from numbers(10000);
|
||||
insert into test select number, toJSONString(map('a.b', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(10000, 10000);
|
||||
insert into test select number, toJSONString(map('a.r', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(20000, 10000);
|
||||
insert into test select number, toJSONString(map('a.a1', number, 'a.a2', number, 'a.a3', number, 'a.a4', number, 'a.a5', number, 'a.a6', number, 'a.a7', number, 'a.a8', number, 'a.r', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(30000, 10000);
|
||||
|
||||
select distinct arrayJoin(JSONAllPathsWithTypes(json)) as paths_with_types from test order by paths_with_types;
|
||||
select distinct arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.b))) as paths_with_types from test order by paths_with_types;
|
||||
select distinct arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.r[]))) as paths_with_types from test order by paths_with_types;
|
||||
|
||||
select json, json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test format Null;
|
||||
select json, json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test order by id format Null;
|
||||
select json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test format Null;
|
||||
select json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test order by id format Null;
|
||||
|
||||
select count() from test where empty(json.a.r[].c.d.e) and empty(json.a.r[].b.c.d_0) and empty(json.a.r[].b.c.d_1);
|
||||
select count() from test where empty(json.a.r[].c.d.e.:`Array(Nullable(Int64))`) and empty(json.a.r[].b.c.d_0.:Int64) and empty(json.a.r[].b.c.d_1.:Int64);
|
||||
select count() from test where arrayJoin(json.a.r[].c.d.e) is null and arrayJoin(json.a.r[].b.c.d_0) is null and arrayJoin(json.a.r[].b.c.d_1) is null;
|
||||
select count() from test where arrayJoin(json.a.r[].c.d.e.:`Array(Nullable(Int64))`) is null and arrayJoin(json.a.r[].b.c.d_0.:Int64) is null and arrayJoin(json.a.r[].b.c.d_1.:Int64) is null;
|
||||
|
||||
select json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test format Null;
|
||||
select json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test order by id format Null;
|
||||
select json.a.r[].c.d.e.:`Array(Nullable(Int64))`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test format Null;
|
||||
select json.a.r[].c.d.e.:`Array(Nullable(Int64))`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test order by id format Null;
|
||||
select json.a.r, json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test format Null;
|
||||
select json.a.r, json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test order by id format Null;
|
||||
select json.a.r, json.a.r[].c.d.e.:`Array(Nullable(Int64))`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test format Null;
|
||||
select json.a.r, json.a.r[].c.d.e.:`Array(Nullable(Int64))`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test order by id format Null;
|
||||
|
||||
select count() from test where empty(json.a.r[].^b) and empty(json.a.r[].^b.c) and empty(json.a.r[].b.c.d_0);
|
||||
select count() from test where empty(json.a.r[].^b) and empty(json.a.r[].^b.c) and empty(json.a.r[].b.c.d_0.:Int64);
|
||||
select count() from test where JSONEmpty(arrayJoin(json.a.r[].^b)) and JSONEmpty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0) is null;
|
||||
select count() from test where JSONEmpty(arrayJoin(json.a.r[].^b)) and JSONEmpty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0.:Int64) is null;
|
||||
|
||||
select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test format Null;
|
||||
select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test order by id format Null;
|
||||
select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test format Null;
|
||||
select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test order by id format Null;
|
||||
select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test format Null;
|
||||
select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test order by id format Null;
|
||||
select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test format Null;
|
||||
select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test order by id format Null;
|
||||
|
||||
drop table test;
|
@ -1,6 +1,3 @@
|
||||
No merges
|
||||
insert
|
||||
test
|
||||
('a.a1','String')
|
||||
('a.a2','String')
|
||||
('a.a3','String')
|
||||
@ -31,8 +28,6 @@ test
|
||||
20000
|
||||
0
|
||||
0
|
||||
With merges
|
||||
test
|
||||
('a.a1','String')
|
||||
('a.a2','String')
|
||||
('a.a3','String')
|
@ -1,73 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest, long, no-debug, no-tsan, no-asan, no-msan, no-ubsan
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# reset --log_comment
|
||||
CLICKHOUSE_LOG_COMMENT=
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_json_type=1 --allow_experimental_variant_type=1 --use_variant_as_common_type=1"
|
||||
|
||||
function insert()
|
||||
{
|
||||
echo "insert"
|
||||
$CH_CLIENT -q "truncate table test"
|
||||
$CH_CLIENT -q "insert into test select number, '{}' from numbers(10000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(10000, 10000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.r', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(20000, 10000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.a1', number, 'a.a2', number, 'a.a3', number, 'a.a4', number, 'a.a5', number, 'a.a6', number, 'a.a7', number, 'a.a8', number, 'a.r', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(30000, 10000)"
|
||||
}
|
||||
|
||||
function test()
|
||||
{
|
||||
echo "test"
|
||||
$CH_CLIENT -q "select distinct arrayJoin(JSONAllPathsWithTypes(json)) as paths_with_types from test order by paths_with_types"
|
||||
$CH_CLIENT -q "select distinct arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.b))) as paths_with_types from test order by paths_with_types"
|
||||
$CH_CLIENT -q "select distinct arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.r[]))) as paths_with_types from test order by paths_with_types"
|
||||
|
||||
$CH_CLIENT -q "select json, json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test format Null"
|
||||
$CH_CLIENT -q "select json, json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test format Null"
|
||||
$CH_CLIENT -q "select json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where empty(json.a.r[].c.d.e) and empty(json.a.r[].b.c.d_0) and empty(json.a.r[].b.c.d_1)"
|
||||
$CH_CLIENT -q "select count() from test where empty(json.a.r[].c.d.e.:\`Array(Nullable(Int64))\`) and empty(json.a.r[].b.c.d_0.:Int64) and empty(json.a.r[].b.c.d_1.:Int64)"
|
||||
$CH_CLIENT -q "select count() from test where arrayJoin(json.a.r[].c.d.e) is null and arrayJoin(json.a.r[].b.c.d_0) is null and arrayJoin(json.a.r[].b.c.d_1) is null"
|
||||
$CH_CLIENT -q "select count() from test where arrayJoin(json.a.r[].c.d.e.:\`Array(Nullable(Int64))\`) is null and arrayJoin(json.a.r[].b.c.d_0.:Int64) is null and arrayJoin(json.a.r[].b.c.d_1.:Int64) is null"
|
||||
|
||||
$CH_CLIENT -q "select json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.r[].c.d.e.:\`Array(Nullable(Int64))\`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r[].c.d.e.:\`Array(Nullable(Int64))\`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].c.d.e.:\`Array(Nullable(Int64))\`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].c.d.e.:\`Array(Nullable(Int64))\`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test order by id format Null"
|
||||
|
||||
$CH_CLIENT -q "select count() from test where empty(json.a.r[].^b) and empty(json.a.r[].^b.c) and empty(json.a.r[].b.c.d_0)"
|
||||
$CH_CLIENT -q "select count() from test where empty(json.a.r[].^b) and empty(json.a.r[].^b.c) and empty(json.a.r[].b.c.d_0.:Int64)"
|
||||
$CH_CLIENT -q "select count() from test where JSONEmpty(arrayJoin(json.a.r[].^b)) and JSONEmpty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0) is null"
|
||||
$CH_CLIENT -q "select count() from test where JSONEmpty(arrayJoin(json.a.r[].^b)) and JSONEmpty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0.:Int64) is null"
|
||||
|
||||
$CH_CLIENT -q "select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test order by id format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test format Null"
|
||||
$CH_CLIENT -q "select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test order by id format Null"
|
||||
}
|
||||
|
||||
$CH_CLIENT -q "drop table if exists test;"
|
||||
|
||||
$CH_CLIENT -q "create table test (id UInt64, json JSON(max_dynamic_paths=8, a.b Array(JSON))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;"
|
||||
echo "No merges"
|
||||
$CH_CLIENT -q "system stop merges test"
|
||||
insert
|
||||
test
|
||||
echo "With merges"
|
||||
$CH_CLIENT -q "system start merges test"
|
||||
test
|
||||
$CH_CLIENT -q "drop table test;"
|
@ -0,0 +1,57 @@
|
||||
-- Tags: no-fasttest, long, no-debug, no-tsan, no-asan, no-msan, no-ubsan
|
||||
|
||||
set allow_experimental_json_type = 1;
|
||||
set allow_experimental_variant_type = 1;
|
||||
set use_variant_as_common_type = 1;
|
||||
|
||||
create table test (id UInt64, json JSON(max_dynamic_paths=8, a.b Array(JSON))) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;
|
||||
|
||||
insert into test select number, '{}' from numbers(10000);
|
||||
insert into test select number, toJSONString(map('a.b', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(10000, 10000);
|
||||
insert into test select number, toJSONString(map('a.r', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(20000, 10000);
|
||||
insert into test select number, toJSONString(map('a.a1', number, 'a.a2', number, 'a.a3', number, 'a.a4', number, 'a.a5', number, 'a.a6', number, 'a.a7', number, 'a.a8', number, 'a.r', arrayMap(x -> map('b.c.d_' || toString(x), number::UInt32, 'c.d.e', range((number + x) % 5 + 1)), range(number % 5 + 1)))) from numbers(30000, 10000);
|
||||
|
||||
{% for merge_command in ['system stop merges test', 'system start merges test'] -%}
|
||||
|
||||
{{ merge_command }};
|
||||
|
||||
select distinct arrayJoin(JSONAllPathsWithTypes(json)) as paths_with_types from test order by paths_with_types;
|
||||
select distinct arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.b))) as paths_with_types from test order by paths_with_types;
|
||||
select distinct arrayJoin(JSONAllPathsWithTypes(arrayJoin(json.a.r[]))) as paths_with_types from test order by paths_with_types;
|
||||
|
||||
select json, json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test format Null;
|
||||
select json, json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test order by id format Null;
|
||||
select json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test format Null;
|
||||
select json.a.b, json.a.b.c, json.a.b.c.d.e, json.a.b.b.c.d_0, json.a.b.b.c.d_1, json.a.b.b.c.d_2, json.a.b.b.c.d_3, json.a.b.b.c.d_4, json.a.r, json.a.r[], json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1, json.a.r[].b.c.d_2, json.a.r[].b.c.d_3, json.a.r[].b.c.d_4, json.^a, json.a.b.^b.c, json.a.r[].^b.c from test order by id format Null;
|
||||
|
||||
select count() from test where empty(json.a.r[].c.d.e) and empty(json.a.r[].b.c.d_0) and empty(json.a.r[].b.c.d_1);
|
||||
select count() from test where empty(json.a.r[].c.d.e.:`Array(Nullable(Int64))`) and empty(json.a.r[].b.c.d_0.:Int64) and empty(json.a.r[].b.c.d_1.:Int64);
|
||||
select count() from test where arrayJoin(json.a.r[].c.d.e) is null and arrayJoin(json.a.r[].b.c.d_0) is null and arrayJoin(json.a.r[].b.c.d_1) is null;
|
||||
select count() from test where arrayJoin(json.a.r[].c.d.e.:`Array(Nullable(Int64))`) is null and arrayJoin(json.a.r[].b.c.d_0.:Int64) is null and arrayJoin(json.a.r[].b.c.d_1.:Int64) is null;
|
||||
|
||||
select json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test format Null;
|
||||
select json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test order by id format Null;
|
||||
select json.a.r[].c.d.e.:`Array(Nullable(Int64))`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test format Null;
|
||||
select json.a.r[].c.d.e.:`Array(Nullable(Int64))`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test order by id format Null;
|
||||
select json.a.r, json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test format Null;
|
||||
select json.a.r, json.a.r[].c.d.e, json.a.r[].b.c.d_0, json.a.r[].b.c.d_1 from test order by id format Null;
|
||||
select json.a.r, json.a.r[].c.d.e.:`Array(Nullable(Int64))`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test format Null;
|
||||
select json.a.r, json.a.r[].c.d.e.:`Array(Nullable(Int64))`, json.a.r[].b.c.d_0.:Int64, json.a.r[].b.c.d_1.:Int64 from test order by id format Null;
|
||||
|
||||
select count() from test where empty(json.a.r[].^b) and empty(json.a.r[].^b.c) and empty(json.a.r[].b.c.d_0);
|
||||
select count() from test where empty(json.a.r[].^b) and empty(json.a.r[].^b.c) and empty(json.a.r[].b.c.d_0.:Int64);
|
||||
select count() from test where JSONEmpty(arrayJoin(json.a.r[].^b)) and JSONEmpty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0) is null;
|
||||
select count() from test where JSONEmpty(arrayJoin(json.a.r[].^b)) and JSONEmpty(arrayJoin(json.a.r[].^b.c)) and arrayJoin(json.a.r[].b.c.d_0.:Int64) is null;
|
||||
|
||||
select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test format Null;
|
||||
select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test order by id format Null;
|
||||
select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test format Null;
|
||||
select json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test order by id format Null;
|
||||
select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test format Null;
|
||||
select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0 from test order by id format Null;
|
||||
select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test format Null;
|
||||
select json.a.r, json.a.r[].^b, json.a.r[].^b.c, json.a.r[].b.c.d_0.:Int64 from test order by id format Null;
|
||||
|
||||
{% endfor -%}
|
||||
|
||||
drop table test;
|
@ -1,5 +1,3 @@
|
||||
MergeTree compact
|
||||
test
|
||||
Dynamic paths
|
||||
100000 a
|
||||
90000 b
|
||||
@ -48,8 +46,6 @@ Shared data paths
|
||||
70000 d
|
||||
60000 e
|
||||
10000 g
|
||||
MergeTree wide
|
||||
test
|
||||
Dynamic paths
|
||||
100000 a
|
||||
90000 b
|
@ -1,70 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest, long, no-debug, no-tsan, no-asan, no-msan, no-ubsan
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# reset --log_comment
|
||||
CLICKHOUSE_LOG_COMMENT=
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
|
||||
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_json_type=1"
|
||||
|
||||
function test()
|
||||
{
|
||||
echo "test"
|
||||
$CH_CLIENT -q "system stop merges test"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a', number)) from numbers(100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('b', number)) from numbers(90000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('c', number)) from numbers(80000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('d', number)) from numbers(70000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('e', number)) from numbers(60000)"
|
||||
$CH_CLIENT -q "insert into test select number, '{}' from numbers(100000)"
|
||||
|
||||
echo "Dynamic paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path"
|
||||
echo "Shared data paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path"
|
||||
$CH_CLIENT -nm -q "system start merges test; optimize table test final;"
|
||||
echo "Dynamic paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path"
|
||||
echo "Shared data paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path"
|
||||
|
||||
$CH_CLIENT -q "system stop merges test"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('f', number)) from numbers(200000)"
|
||||
echo "Dynamic paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path"
|
||||
echo "Shared data paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path"
|
||||
$CH_CLIENT -nm -q "system start merges test; optimize table test final;"
|
||||
echo "Dynamic paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path"
|
||||
echo "Shared data paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path"
|
||||
|
||||
$CH_CLIENT -q "system stop merges test"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('g', number)) from numbers(10000)"
|
||||
echo "Dynamic paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path"
|
||||
echo "Shared data paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path"
|
||||
$CH_CLIENT -nm -q "system start merges test; optimize table test final;"
|
||||
echo "Dynamic paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path"
|
||||
echo "Shared data paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path"
|
||||
}
|
||||
|
||||
$CH_CLIENT -q "drop table if exists test;"
|
||||
|
||||
echo "MergeTree compact"
|
||||
$CH_CLIENT -q "create table test (id UInt64, json JSON(max_dynamic_paths=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;"
|
||||
test
|
||||
$CH_CLIENT -q "drop table test;"
|
||||
|
||||
echo "MergeTree wide"
|
||||
$CH_CLIENT -q "create table test (id UInt64, json JSON(max_dynamic_paths=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;"
|
||||
test
|
||||
$CH_CLIENT -q "drop table test;"
|
||||
|
@ -0,0 +1,59 @@
|
||||
-- Tags: no-fasttest, long, no-debug, no-tsan, no-asan, no-msan, no-ubsan
|
||||
|
||||
set allow_experimental_json_type = 1;
|
||||
|
||||
drop table if exists test;
|
||||
|
||||
{% for create_command in ['create table test (id UInt64, json JSON(max_dynamic_paths=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;',
|
||||
'create table test (id UInt64, json JSON(max_dynamic_paths=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=10, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;'] -%}
|
||||
|
||||
{{ create_command }}
|
||||
|
||||
system stop merges test;
|
||||
insert into test select number, toJSONString(map('a', number)) from numbers(100000);
|
||||
insert into test select number, toJSONString(map('b', number)) from numbers(90000);
|
||||
insert into test select number, toJSONString(map('c', number)) from numbers(80000);
|
||||
insert into test select number, toJSONString(map('d', number)) from numbers(70000);
|
||||
insert into test select number, toJSONString(map('e', number)) from numbers(60000);
|
||||
insert into test select number, '{}' from numbers(100000);
|
||||
|
||||
select 'Dynamic paths';
|
||||
select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path;
|
||||
select 'Shared data paths';
|
||||
select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path;
|
||||
system start merges test;
|
||||
optimize table test final;
|
||||
select 'Dynamic paths';
|
||||
select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path;
|
||||
select 'Shared data paths';
|
||||
select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path;
|
||||
|
||||
system stop merges test;
|
||||
insert into test select number, toJSONString(map('f', number)) from numbers(200000);
|
||||
select 'Dynamic paths';
|
||||
select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path;
|
||||
select 'Shared data paths';
|
||||
select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path;
|
||||
system start merges test;
|
||||
optimize table test final;
|
||||
select 'Dynamic paths';
|
||||
select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path;
|
||||
select 'Shared data paths';
|
||||
select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path;
|
||||
|
||||
system stop merges test;
|
||||
insert into test select number, toJSONString(map('g', number)) from numbers(10000);
|
||||
select 'Dynamic paths';
|
||||
select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path;
|
||||
select 'Shared data paths';
|
||||
select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path;
|
||||
system start merges test;
|
||||
optimize table test final;
|
||||
select 'Dynamic paths';
|
||||
select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path;
|
||||
select 'Shared data paths';
|
||||
select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path;
|
||||
|
||||
drop table test;
|
||||
|
||||
{% endfor -%}
|
@ -1,5 +1,3 @@
|
||||
MergeTree compact
|
||||
test
|
||||
Dynamic paths
|
||||
100000 a
|
||||
90000 b
|
||||
@ -48,8 +46,6 @@ Shared data paths
|
||||
70000 d
|
||||
60000 e
|
||||
10000 g
|
||||
MergeTree wide
|
||||
test
|
||||
Dynamic paths
|
||||
100000 a
|
||||
90000 b
|
@ -1,70 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest, long, no-debug, no-tsan, no-asan, no-msan, no-ubsan
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# reset --log_comment
|
||||
CLICKHOUSE_LOG_COMMENT=
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
|
||||
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_json_type=1"
|
||||
|
||||
function test()
|
||||
{
|
||||
echo "test"
|
||||
$CH_CLIENT -q "system stop merges test"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a', number)) from numbers(100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('b', number)) from numbers(90000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('c', number)) from numbers(80000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('d', number)) from numbers(70000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('e', number)) from numbers(60000)"
|
||||
$CH_CLIENT -q "insert into test select number, '{}' from numbers(100000)"
|
||||
|
||||
echo "Dynamic paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path"
|
||||
echo "Shared data paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path"
|
||||
$CH_CLIENT -nm -q "system start merges test; optimize table test final;"
|
||||
echo "Dynamic paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path"
|
||||
echo "Shared data paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path"
|
||||
|
||||
$CH_CLIENT -q "system stop merges test"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('f', number)) from numbers(200000)"
|
||||
echo "Dynamic paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path"
|
||||
echo "Shared data paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path"
|
||||
$CH_CLIENT -nm -q "system start merges test; optimize table test final;"
|
||||
echo "Dynamic paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path"
|
||||
echo "Shared data paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path"
|
||||
|
||||
$CH_CLIENT -q "system stop merges test"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('g', number)) from numbers(10000)"
|
||||
echo "Dynamic paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path"
|
||||
echo "Shared data paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path"
|
||||
$CH_CLIENT -nm -q "system start merges test; optimize table test final;"
|
||||
echo "Dynamic paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path"
|
||||
echo "Shared data paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path"
|
||||
}
|
||||
|
||||
$CH_CLIENT -q "drop table if exists test;"
|
||||
|
||||
echo "MergeTree compact"
|
||||
$CH_CLIENT -q "create table test (id UInt64, json JSON(max_dynamic_paths=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;"
|
||||
test
|
||||
$CH_CLIENT -q "drop table test;"
|
||||
|
||||
echo "MergeTree wide"
|
||||
$CH_CLIENT -q "create table test (id UInt64, json JSON(max_dynamic_paths=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;"
|
||||
test
|
||||
$CH_CLIENT -q "drop table test;"
|
||||
|
@ -0,0 +1,59 @@
|
||||
-- Tags: no-fasttest, long, no-debug, no-tsan, no-asan, no-msan, no-ubsan
|
||||
|
||||
set allow_experimental_json_type = 1;
|
||||
|
||||
drop table if exists test;
|
||||
|
||||
{% for create_command in ['create table test (id UInt64, json JSON(max_dynamic_paths=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;',
|
||||
'create table test (id UInt64, json JSON(max_dynamic_paths=3)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_columns_to_activate=1, index_granularity_bytes=10485760, index_granularity=8192, merge_max_block_size=8192, merge_max_block_size_bytes=10485760;'] -%}
|
||||
|
||||
{{ create_command }}
|
||||
|
||||
system stop merges test;
|
||||
insert into test select number, toJSONString(map('a', number)) from numbers(100000);
|
||||
insert into test select number, toJSONString(map('b', number)) from numbers(90000);
|
||||
insert into test select number, toJSONString(map('c', number)) from numbers(80000);
|
||||
insert into test select number, toJSONString(map('d', number)) from numbers(70000);
|
||||
insert into test select number, toJSONString(map('e', number)) from numbers(60000);
|
||||
insert into test select number, '{}' from numbers(100000);
|
||||
|
||||
select 'Dynamic paths';
|
||||
select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path;
|
||||
select 'Shared data paths';
|
||||
select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path;
|
||||
system start merges test;
|
||||
optimize table test final;
|
||||
select 'Dynamic paths';
|
||||
select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path;
|
||||
select 'Shared data paths';
|
||||
select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path;
|
||||
|
||||
system stop merges test;
|
||||
insert into test select number, toJSONString(map('f', number)) from numbers(200000);
|
||||
select 'Dynamic paths';
|
||||
select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path;
|
||||
select 'Shared data paths';
|
||||
select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path;
|
||||
system start merges test;
|
||||
optimize table test final;
|
||||
select 'Dynamic paths';
|
||||
select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path;
|
||||
select 'Shared data paths';
|
||||
select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path;
|
||||
|
||||
system stop merges test;
|
||||
insert into test select number, toJSONString(map('g', number)) from numbers(10000);
|
||||
select 'Dynamic paths';
|
||||
select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path;
|
||||
select 'Shared data paths';
|
||||
select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path;
|
||||
system start merges test;
|
||||
optimize table test final;
|
||||
select 'Dynamic paths';
|
||||
select count(), arrayJoin(JSONDynamicPaths(json)) as path from test group by path order by count() desc, path;
|
||||
select 'Shared data paths';
|
||||
select count(), arrayJoin(JSONSharedDataPaths(json)) as path from test group by path order by count() desc, path;
|
||||
|
||||
drop table test;
|
||||
|
||||
{% endfor -%}
|
@ -1,4 +1,3 @@
|
||||
Memory
|
||||
initial insert
|
||||
alter add column 1
|
||||
0 {} \N {} \N \N
|
||||
@ -23,7 +22,6 @@ insert after alter add column
|
||||
12 {} \N {} \N \N
|
||||
13 {} \N {} \N \N
|
||||
14 {} \N {} \N \N
|
||||
MergeTree compact
|
||||
initial insert
|
||||
alter add column 1
|
||||
0 {} \N {} \N \N
|
||||
@ -48,7 +46,6 @@ insert after alter add column
|
||||
12 {} \N {} \N \N
|
||||
13 {} \N {} \N \N
|
||||
14 {} \N {} \N \N
|
||||
MergeTree wide
|
||||
initial insert
|
||||
alter add column 1
|
||||
0 {} \N {} \N \N
|
@ -1,47 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest, long
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# reset --log_comment
|
||||
CLICKHOUSE_LOG_COMMENT=
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_dynamic_type=1 --allow_experimental_variant_type=1 --use_variant_as_common_type=1"
|
||||
|
||||
function run()
|
||||
{
|
||||
echo "initial insert"
|
||||
$CH_CLIENT -q "insert into test select number from numbers(3)"
|
||||
|
||||
echo "alter add column 1"
|
||||
$CH_CLIENT -q "alter table test add column json JSON settings mutations_sync=1"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONAllPaths(json)) as path from test group by path order by count() desc, path"
|
||||
$CH_CLIENT -q "select x, json, json.a.b, json.^a, json.b.c.:Int64, json.c.d from test order by x"
|
||||
|
||||
echo "insert after alter add column"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a.b', number::UInt32)) from numbers(3, 3)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('b.c', number::UInt32)) from numbers(6, 3)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('c.d', number::UInt32)) from numbers(9, 3)"
|
||||
$CH_CLIENT -q "insert into test select number, '{}' from numbers(12, 3)"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONAllPaths(json)) as path from test group by path order by count() desc, path"
|
||||
$CH_CLIENT -q "select x, json, json.a.b, json.^a, json.b.c.:Int64, json.c.d from test order by x"
|
||||
}
|
||||
|
||||
$CH_CLIENT -q "drop table if exists test;"
|
||||
|
||||
echo "Memory"
|
||||
$CH_CLIENT -q "create table test (x UInt64) engine=Memory"
|
||||
run
|
||||
$CH_CLIENT -q "drop table test;"
|
||||
|
||||
echo "MergeTree compact"
|
||||
$CH_CLIENT -q "create table test (x UInt64) engine=MergeTree order by x settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;"
|
||||
run
|
||||
$CH_CLIENT -q "drop table test;"
|
||||
|
||||
echo "MergeTree wide"
|
||||
$CH_CLIENT -q "create table test (x UInt64) engine=MergeTree order by x settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;"
|
||||
run
|
||||
$CH_CLIENT -q "drop table test;"
|
||||
|
@ -0,0 +1,33 @@
|
||||
-- Tags: no-fasttest, long
|
||||
|
||||
set allow_experimental_dynamic_type = 1;
|
||||
set allow_experimental_variant_type = 1;
|
||||
set use_variant_as_common_type = 1;
|
||||
|
||||
drop table if exists test;
|
||||
|
||||
{% for create_command in ['create table test (x UInt64) engine=Memory;',
|
||||
'create table test (x UInt64) engine=MergeTree order by x settings min_rows_for_wide_part=100000000, min_bytes_for_wide_part=1000000000;',
|
||||
'create table test (x UInt64) engine=MergeTree order by x settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;'] -%}
|
||||
|
||||
{{ create_command }}
|
||||
|
||||
select 'initial insert';
|
||||
insert into test select number from numbers(3);
|
||||
|
||||
select 'alter add column 1';
|
||||
alter table test add column json JSON settings mutations_sync=1;
|
||||
select count(), arrayJoin(JSONAllPaths(json)) as path from test group by path order by count() desc, path;
|
||||
select x, json, json.a.b, json.^a, json.b.c.:Int64, json.c.d from test order by x;
|
||||
|
||||
select 'insert after alter add column';
|
||||
insert into test select number, toJSONString(map('a.b', number::UInt32)) from numbers(3, 3);
|
||||
insert into test select number, toJSONString(map('b.c', number::UInt32)) from numbers(6, 3);
|
||||
insert into test select number, toJSONString(map('c.d', number::UInt32)) from numbers(9, 3);
|
||||
insert into test select number, '{}' from numbers(12, 3);
|
||||
select count(), arrayJoin(JSONAllPaths(json)) as path from test group by path order by count() desc, path;
|
||||
select x, json, json.a.b, json.^a, json.b.c.:Int64, json.c.d from test order by x;
|
||||
|
||||
drop table test;
|
||||
|
||||
{% endfor -%}
|
@ -1,5 +1,3 @@
|
||||
MergeTree compact + horizontal merge
|
||||
test
|
||||
Dynamic paths
|
||||
300000 c
|
||||
150000 d
|
||||
@ -20,8 +18,6 @@ Shared data paths
|
||||
300000 c
|
||||
150000 d
|
||||
150000 e
|
||||
MergeTree wide + horizontal merge
|
||||
test
|
||||
Dynamic paths
|
||||
300000 c
|
||||
150000 d
|
||||
@ -42,8 +38,6 @@ Shared data paths
|
||||
300000 c
|
||||
150000 d
|
||||
150000 e
|
||||
MergeTree compact + vertical merge
|
||||
test
|
||||
Dynamic paths
|
||||
300000 c
|
||||
150000 d
|
||||
@ -64,8 +58,6 @@ Shared data paths
|
||||
300000 c
|
||||
150000 d
|
||||
150000 e
|
||||
MergeTree wide + vertical merge
|
||||
test
|
||||
Dynamic paths
|
||||
300000 c
|
||||
150000 d
|
@ -1,65 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest, long, no-debug, no-tsan, no-asan, no-msan, no-ubsan
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# reset --log_comment
|
||||
CLICKHOUSE_LOG_COMMENT=
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
CH_CLIENT="$CLICKHOUSE_CLIENT --allow_experimental_json_type=1"
|
||||
|
||||
|
||||
function test()
|
||||
{
|
||||
echo "test"
|
||||
$CH_CLIENT -q "system stop merges test"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('a', number)) from numbers(100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('b', arrayMap(x -> map('c', x), range(number % 5 + 1)))) from numbers(100000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('b', arrayMap(x -> map('d', x), range(number % 5 + 1)))) from numbers(50000)"
|
||||
|
||||
echo "Dynamic paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONDynamicPaths(arrayJoin(json.b[]))) as path from test group by path order by count() desc, path"
|
||||
echo "Shared data paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONSharedDataPaths(arrayJoin(json.b[]))) as path from test group by path order by count() desc, path"
|
||||
$CH_CLIENT -nm -q "system start merges test; optimize table test final;"
|
||||
echo "Dynamic paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONDynamicPaths(arrayJoin(json.b[]))) as path from test group by path order by count() desc, path"
|
||||
echo "Shared data paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONSharedDataPaths(arrayJoin(json.b[]))) as path from test group by path order by count() desc, path"
|
||||
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('b', arrayMap(x -> map('e', x), range(number % 5 + 1)))) from numbers(50000)"
|
||||
$CH_CLIENT -q "insert into test select number, toJSONString(map('b', arrayMap(x -> map('f', x), range(number % 5 + 1)))) from numbers(200000)"
|
||||
|
||||
echo "Dynamic paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONDynamicPaths(arrayJoin(json.b[]))) as path from test group by path order by count() desc, path"
|
||||
echo "Shared data paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONSharedDataPaths(arrayJoin(json.b[]))) as path from test group by path order by count() desc, path"
|
||||
$CH_CLIENT -nm -q "system start merges test; optimize table test final;"
|
||||
echo "Dynamic paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONDynamicPaths(arrayJoin(json.b[]))) as path from test group by path order by count() desc, path"
|
||||
echo "Shared data paths"
|
||||
$CH_CLIENT -q "select count(), arrayJoin(JSONSharedDataPaths(arrayJoin(json.b[]))) as path from test group by path order by count() desc, path"
|
||||
}
|
||||
|
||||
$CH_CLIENT -q "drop table if exists test;"
|
||||
|
||||
echo "MergeTree compact + horizontal merge"
|
||||
$CH_CLIENT -q "create table test (id UInt64, json JSON(max_dynamic_paths=4)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;"
|
||||
test
|
||||
$CH_CLIENT -q "drop table test;"
|
||||
|
||||
echo "MergeTree wide + horizontal merge"
|
||||
$CH_CLIENT -q "create table test (id UInt64, json JSON(max_dynamic_paths=4)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;"
|
||||
test
|
||||
$CH_CLIENT -q "drop table test;"
|
||||
|
||||
echo "MergeTree compact + vertical merge"
|
||||
$CH_CLIENT -q "create table test (id UInt64, json JSON(max_dynamic_paths=4)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;"
|
||||
test
|
||||
$CH_CLIENT -q "drop table test;"
|
||||
|
||||
echo "MergeTree wide + vertical merge"
|
||||
$CH_CLIENT -q "create table test (id UInt64, json JSON(max_dynamic_paths=4)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;"
|
||||
test
|
||||
$CH_CLIENT -q "drop table test;"
|
46
tests/queries/0_stateless/03211_nested_json_merges.sql.j2
Normal file
46
tests/queries/0_stateless/03211_nested_json_merges.sql.j2
Normal file
@ -0,0 +1,46 @@
|
||||
-- Tags: no-fasttest, long, no-debug, no-tsan, no-asan, no-msan, no-ubsan
|
||||
|
||||
set allow_experimental_json_type = 1;
|
||||
|
||||
drop table if exists test;
|
||||
|
||||
{% for create_command in ['create table test (id UInt64, json JSON(max_dynamic_paths=4)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000;',
|
||||
'create table test (id UInt64, json JSON(max_dynamic_paths=4)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1;',
|
||||
'create table test (id UInt64, json JSON(max_dynamic_paths=4)) engine=MergeTree order by id settings min_rows_for_wide_part=1000000000, min_bytes_for_wide_part=10000000000, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;',
|
||||
'create table test (id UInt64, json JSON(max_dynamic_paths=4)) engine=MergeTree order by id settings min_rows_for_wide_part=1, min_bytes_for_wide_part=1, vertical_merge_algorithm_min_rows_to_activate=1, vertical_merge_algorithm_min_columns_to_activate=1;'] -%}
|
||||
|
||||
{{ create_command }}
|
||||
|
||||
system stop merges test;
|
||||
insert into test select number, toJSONString(map('a', number)) from numbers(100000);
|
||||
insert into test select number, toJSONString(map('b', arrayMap(x -> map('c', x), range(number % 5 + 1)))) from numbers(100000);
|
||||
insert into test select number, toJSONString(map('b', arrayMap(x -> map('d', x), range(number % 5 + 1)))) from numbers(50000);
|
||||
|
||||
select 'Dynamic paths';
|
||||
select count(), arrayJoin(JSONDynamicPaths(arrayJoin(json.b[]))) as path from test group by path order by count() desc, path;
|
||||
select 'Shared data paths';
|
||||
select count(), arrayJoin(JSONSharedDataPaths(arrayJoin(json.b[]))) as path from test group by path order by count() desc, path;
|
||||
system start merges test;
|
||||
optimize table test final;
|
||||
select 'Dynamic paths';
|
||||
select count(), arrayJoin(JSONDynamicPaths(arrayJoin(json.b[]))) as path from test group by path order by count() desc, path;
|
||||
select 'Shared data paths';
|
||||
select count(), arrayJoin(JSONSharedDataPaths(arrayJoin(json.b[]))) as path from test group by path order by count() desc, path;
|
||||
|
||||
insert into test select number, toJSONString(map('b', arrayMap(x -> map('e', x), range(number % 5 + 1)))) from numbers(50000);
|
||||
insert into test select number, toJSONString(map('b', arrayMap(x -> map('f', x), range(number % 5 + 1)))) from numbers(200000);
|
||||
|
||||
select 'Dynamic paths';
|
||||
select count(), arrayJoin(JSONDynamicPaths(arrayJoin(json.b[]))) as path from test group by path order by count() desc, path;
|
||||
select 'Shared data paths';
|
||||
select count(), arrayJoin(JSONSharedDataPaths(arrayJoin(json.b[]))) as path from test group by path order by count() desc, path;
|
||||
system start merges test;
|
||||
optimize table test final;
|
||||
select 'Dynamic paths';
|
||||
select count(), arrayJoin(JSONDynamicPaths(arrayJoin(json.b[]))) as path from test group by path order by count() desc, path;
|
||||
select 'Shared data paths';
|
||||
select count(), arrayJoin(JSONSharedDataPaths(arrayJoin(json.b[]))) as path from test group by path order by count() desc, path;
|
||||
|
||||
drop table test;
|
||||
|
||||
{% endfor -%}
|
Loading…
Reference in New Issue
Block a user